diff --git a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java index 32426076b7e1c..f18c1c2154225 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java @@ -14,6 +14,7 @@ package io.trino.sql.query; import com.google.common.base.Joiner; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.errorprone.annotations.CheckReturnValue; @@ -40,9 +41,15 @@ import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; import io.trino.testing.StandaloneQueryRunner; +import io.trino.testing.assertions.TrinoExceptionAssert; import org.assertj.core.api.AbstractAssert; +import org.assertj.core.api.AbstractCollectionAssert; +import org.assertj.core.api.AbstractIntegerAssert; +import org.assertj.core.api.AbstractThrowableAssert; import org.assertj.core.api.AssertProvider; +import org.assertj.core.api.Descriptable; import org.assertj.core.api.ListAssert; +import org.assertj.core.api.ObjectAssert; import org.assertj.core.description.Description; import org.assertj.core.description.TextDescription; import org.assertj.core.presentation.Representation; @@ -52,6 +59,7 @@ import java.io.Closeable; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,9 +67,12 @@ import java.util.Optional; import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Supplier; import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Suppliers.memoize; import static com.google.common.collect.Iterables.getOnlyElement; import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static io.trino.cost.StatsCalculator.noopStatsCalculator; @@ -71,9 +82,12 @@ import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.testing.TransactionBuilder.transaction; +import static io.trino.testing.assertions.TrinoExceptionAssert.assertThatTrinoException; +import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; @@ -256,58 +270,25 @@ protected void executeExclusively(Runnable executionBlock) } public static class QueryAssert - extends AbstractAssert + implements Descriptable { - private static final Representation ROWS_REPRESENTATION = new StandardRepresentation() - { - @Override - public String toStringOf(Object object) - { - if (object instanceof List list) { - return list.stream() - .map(this::toStringOf) - .collect(Collectors.joining(", ")); - } - if (object instanceof MaterializedRow row) { - return row.getFields().stream() - .map(this::formatRowElement) - .collect(Collectors.joining(", ", "(", ")")); - } - return super.toStringOf(object); - } - - private String formatRowElement(Object value) - { - if (value == null) { - return "null"; - } - if (value.getClass().isArray()) { - return formatArray(value); - } - // Using super.toStringOf would add quotes around String values, which could be expected for varchar values - // but would be misleading for date/time values which come as String too. More proper formatting would need to be - // type-aware. - return String.valueOf(value); - } - }; - private final QueryRunner runner; private final Session session; private final Optional query; - private final Description description; + private Description description; + private final Supplier result; private boolean ordered; private boolean skipTypesCheck; private boolean skipResultsCorrectnessCheckForPushdown; static AssertProvider newQueryAssert(String query, QueryRunner runner, Session session) { - MaterializedResult result = runner.execute(session, query); return () -> new QueryAssert( runner, session, Optional.of(query), new TextDescription("%s", query), - result, + Optional.empty(), false, false, false); @@ -318,21 +299,37 @@ private QueryAssert( Session session, Optional query, Description description, - MaterializedResult actual, + Optional result, boolean ordered, boolean skipTypesCheck, boolean skipResultsCorrectnessCheckForPushdown) { - super(actual, Object.class); this.runner = requireNonNull(runner, "runner is null"); this.session = requireNonNull(session, "session is null"); this.query = requireNonNull(query, "query is null"); this.description = requireNonNull(description, "description is null"); + checkArgument(result.isPresent() || query.isPresent(), "Query must be present when result is empty"); + this.result = result + .map(Suppliers::ofInstance) + .orElseGet(() -> memoize(() -> runner.execute(session, query.orElseThrow()))); this.ordered = ordered; this.skipTypesCheck = skipTypesCheck; this.skipResultsCorrectnessCheckForPushdown = skipResultsCorrectnessCheckForPushdown; } + @Override + public QueryAssert describedAs(Description description) + { + this.description = requireNonNull(description, "description is null"); + return this; + } + + /** + * @deprecated use {@code result().exceptColumns(...)} instead. + */ + @Deprecated + @CheckReturnValue + // TODO when exceptColumns(), projected() are removed, simplify constructor public QueryAssert exceptColumns(String... columnNamesToExclude) { return new QueryAssert( @@ -340,12 +337,18 @@ public QueryAssert exceptColumns(String... columnNamesToExclude) session, Optional.empty(), // original query would not produce projected result new TextDescription("%s except columns %s", description, Arrays.toString(columnNamesToExclude)), - actual.exceptColumns(columnNamesToExclude), + Optional.of(result.get().exceptColumns(columnNamesToExclude)), ordered, skipTypesCheck, skipResultsCorrectnessCheckForPushdown); } + /** + * @deprecated use {@code result().projected(...)} instead. + */ + @Deprecated + @CheckReturnValue + // TODO when exceptColumns(), projected() are removed, simplify constructor public QueryAssert projected(String... columnNamesToInclude) { return new QueryAssert( @@ -353,21 +356,16 @@ public QueryAssert projected(String... columnNamesToInclude) session, Optional.empty(), // original query would not produce projected result new TextDescription("%s projected with %s", description, Arrays.toString(columnNamesToInclude)), - actual.project(columnNamesToInclude), + Optional.of(result.get().project(columnNamesToInclude)), ordered, skipTypesCheck, skipResultsCorrectnessCheckForPushdown); } - public QueryAssert matches(BiFunction evaluator) - { - MaterializedResult expected = evaluator.apply(session, runner); - return matches(expected); - } - public QueryAssert succeeds() { - return satisfies(actual -> {}); + MaterializedResult ignored = result.get(); + return this; } public QueryAssert ordered() @@ -391,29 +389,19 @@ public QueryAssert skipResultsCorrectnessCheckForPushdown() @CanIgnoreReturnValue public QueryAssert matches(@Language("SQL") String query) { - MaterializedResult expected = runner.execute(session, query); - return matches(expected); + result().matches(query); + return this; } + /** + * @deprecated use {@code result().matches(...)} instead. + */ + @Deprecated @CanIgnoreReturnValue public QueryAssert matches(MaterializedResult expected) { - return satisfies(actual -> { - if (!skipTypesCheck) { - assertTypes(description, actual, expected.getTypes()); - } - - ListAssert assertion = assertThat(actual.getMaterializedRows()) - .as("Rows for query [%s]", description) - .withRepresentation(ROWS_REPRESENTATION); - - if (ordered) { - assertion.containsExactlyElementsOf(expected.getMaterializedRows()); - } - else { - assertion.containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); - } - }); + result().matches(expected); + return this; } @CanIgnoreReturnValue @@ -438,55 +426,80 @@ public QueryAssert matches(PlanMatchPattern expectedPlan) public QueryAssert containsAll(@Language("SQL") String query) { MaterializedResult expected = runner.execute(session, query); - return containsAll(expected); + result().containsAll(expected); + return this; } + /** + * @deprecated use {@code result().hasType(...)} instead. + */ + @Deprecated @CanIgnoreReturnValue - public QueryAssert containsAll(MaterializedResult expected) + public QueryAssert hasOutputTypes(List expectedTypes) { - return satisfies(actual -> { - if (!skipTypesCheck) { - assertTypes(description, actual, expected.getTypes()); - } - - assertThat(actual.getMaterializedRows()) - .as("Rows for query [%s]", description) - .withRepresentation(ROWS_REPRESENTATION) - .containsAll(expected.getMaterializedRows()); - }); + result().hasTypes(expectedTypes); + return this; } + /** + * @deprecated use {@code result().hasType(...)} instead. + */ + @Deprecated @CanIgnoreReturnValue - public QueryAssert hasOutputTypes(List expectedTypes) + public QueryAssert outputHasType(int index, Type expectedType) { - return satisfies(actual -> { - assertTypes(description, actual, expectedTypes); - }); + result().hasType(index, expectedType); + return this; } + /** + * @deprecated use {@code result().isEmpty()} instead. + */ + @Deprecated @CanIgnoreReturnValue - public QueryAssert outputHasType(int index, Type expectedType) + public QueryAssert returnsEmptyResult() { - return satisfies(actual -> { - assertThat(actual.getTypes()) - .as("Output types for query [%s]", description) - .element(index).isEqualTo(expectedType); - }); + result().isEmpty(); + return this; } - private static void assertTypes(Description queryDescription, MaterializedResult actual, List expectedTypes) + /** + * @see #nonTrinoExceptionFailure() + */ + @CheckReturnValue + public TrinoExceptionAssert failure() { - assertThat(actual.getTypes()) - .as("Output types for query [%s]", queryDescription) - .isEqualTo(expectedTypes); + // TODO provide useful exception message when query does not fail + return assertTrinoExceptionThrownBy(result::get); } - @CanIgnoreReturnValue - public QueryAssert returnsEmptyResult() + /** + * Escape hatch for failures which are (incorrectly) not {@link io.trino.spi.TrinoException} and therefore {@link #failure()} cannot be used. + * + * @deprecated Any need to use this method indicates a bug in the code under test (wrong error reporting). There is no intention to remove this method. + */ + @Deprecated(forRemoval = false) + @CheckReturnValue + public AbstractThrowableAssert nonTrinoExceptionFailure() { - return satisfies(actual -> { - assertThat(actual.getMaterializedRows()).as("Rows for query [%s]", description).isEmpty(); - }); + // TODO provide useful exception message when query does not fail + return assertThatThrownBy(result::get) + .satisfies(throwable -> { + assertThatThrownBy(() -> assertThatTrinoException(throwable)) + .hasMessageStartingWith("Expected TrinoException or wrapper, but got: "); + }); + } + + @CheckReturnValue + public ResultAssert result() + { + return new ResultAssert( + runner, + session, + description, + result.get(), + ordered, + skipTypesCheck); } /** @@ -658,7 +671,7 @@ public QueryAssert hasCorrectResultsRegardlessOfPushdown() Session withoutPushdown = Session.builder(session) .setSystemProperty("allow_pushdown_into_connectors", "false") .build(); - matches(runner.execute(withoutPushdown, query())); + result().matches(runner.execute(withoutPushdown, query())); return this; } @@ -668,6 +681,181 @@ private String query() } } + public static class ResultAssert + extends AbstractAssert + { + private static final Representation ROWS_REPRESENTATION = new StandardRepresentation() + { + @Override + public String toStringOf(Object object) + { + if (object instanceof List list) { + return list.stream() + .map(this::toStringOf) + .collect(Collectors.joining(", ")); + } + if (object instanceof MaterializedRow row) { + return row.getFields().stream() + .map(this::formatRowElement) + .collect(Collectors.joining(", ", "(", ")")); + } + return super.toStringOf(object); + } + + private String formatRowElement(Object value) + { + if (value == null) { + return "null"; + } + if (value.getClass().isArray()) { + return formatArray(value); + } + // Using super.toStringOf would add quotes around String values, which could be expected for varchar values + // but would be misleading for date/time values which come as String too. More proper formatting would need to be + // type-aware. + return String.valueOf(value); + } + }; + + private final QueryRunner runner; + private final Session session; + private final Description description; + private final boolean ordered; + private boolean skipTypesCheck; + + private ResultAssert( + QueryRunner runner, + Session session, + Description description, + MaterializedResult result, + boolean ordered, + boolean skipTypesCheck) + { + super(result, ResultAssert.class); + this.runner = requireNonNull(runner, "runner is null"); + this.session = requireNonNull(session, "session is null"); + this.description = requireNonNull(description, "description is null"); + this.ordered = ordered; + this.skipTypesCheck = skipTypesCheck; + } + + public ResultAssert skippingTypesCheck() + { + this.skipTypesCheck = true; + return this; + } + + public ResultAssert exceptColumns(String... columnNamesToExclude) + { + return new ResultAssert( + runner, + session, + new TextDescription("%s except columns %s", description, Arrays.toString(columnNamesToExclude)), + actual.exceptColumns(columnNamesToExclude), + ordered, + skipTypesCheck); + } + + public ResultAssert projected(String... columnNamesToInclude) + { + return new ResultAssert( + runner, + session, + new TextDescription("%s projected with %s", description, Arrays.toString(columnNamesToInclude)), + actual.project(columnNamesToInclude), + ordered, + skipTypesCheck); + } + + @CanIgnoreReturnValue + public ResultAssert isEmpty() + { + rows().isEmpty(); + return this; + } + + public AbstractIntegerAssert rowCount() + { + return assertThat(actual.getRowCount()) + .as("Row count for query [%s]", description); + } + + @CanIgnoreReturnValue + public ResultAssert matches(@Language("SQL") String query) + { + MaterializedResult expected = runner.execute(session, query); + return matches(expected); + } + + @CanIgnoreReturnValue + public ResultAssert matches(MaterializedResult expected) + { + return satisfies(actual -> { + if (!skipTypesCheck) { + hasTypes(expected.getTypes()); + } + + ListAssert assertion = assertThat(actual.getMaterializedRows()) + .as("Rows for query [%s]", description) + .withRepresentation(ROWS_REPRESENTATION); + + if (ordered) { + assertion.containsExactlyElementsOf(expected.getMaterializedRows()); + } + else { + assertion.containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); + } + }); + } + + @CanIgnoreReturnValue + public ResultAssert containsAll(MaterializedResult expected) + { + return satisfies(actual -> { + if (!skipTypesCheck) { + hasTypes(expected.getTypes()); + } + + assertThat(actual.getMaterializedRows()) + .as("Rows for query [%s]", description) + .withRepresentation(ROWS_REPRESENTATION) + .containsAll(expected.getMaterializedRows()); + }); + } + + @CanIgnoreReturnValue + public ResultAssert hasTypes(List expectedTypes) + { + assertThat(actual.getTypes()) + .as("Output types for query [%s]", description) + .isEqualTo(expectedTypes); + return this; + } + + @CanIgnoreReturnValue + public ResultAssert hasType(int index, Type expectedType) + { + assertThat(actual.getTypes()) + .as("Output types for query [%s]", description) + .element(index).isEqualTo(expectedType); + return this; + } + + public AbstractCollectionAssert, Object, ObjectAssert> onlyColumnAsSet() + { + return assertThat(actual.getOnlyColumnAsSet()) + .as("Only column for query [%s]", description) + .withRepresentation(ROWS_REPRESENTATION); + } + + public ListAssert rows() + { + return assertThat(actual.getMaterializedRows()) + .as("Rows for query [%s]", description) + .withRepresentation(ROWS_REPRESENTATION); + } + } + public static class ExpressionAssertProvider implements AssertProvider { diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestAggregation.java b/core/trino-main/src/test/java/io/trino/sql/query/TestAggregation.java index 4cbc28f8bbbdd..db58b5089afcf 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestAggregation.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestAggregation.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -38,14 +37,14 @@ public void teardown() @Test public void testQuantifiedComparison() { - assertThatThrownBy(() -> assertions.query("SELECT v > ALL (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) - .hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); + assertThat(assertions.query("SELECT v > ALL (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) + .failure().hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); - assertThatThrownBy(() -> assertions.query("SELECT v > ANY (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) - .hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); + assertThat(assertions.query("SELECT v > ANY (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) + .failure().hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); - assertThatThrownBy(() -> assertions.query("SELECT v > SOME (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) - .hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); + assertThat(assertions.query("SELECT v > SOME (VALUES 1) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) + .failure().hasMessageContaining("must be an aggregate expression or appear in GROUP BY clause"); assertThat(assertions.query("SELECT count_if(v > ALL (VALUES 0, 1)) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) .matches("VALUES BIGINT '1'"); @@ -53,7 +52,7 @@ public void testQuantifiedComparison() assertThat(assertions.query("SELECT count_if(v > ANY (VALUES 0, 1)) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) .matches("VALUES BIGINT '2'"); - assertThatThrownBy(() -> assertions.query("SELECT 1 > ALL (VALUES k) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) - .hasMessageContaining("line 1:17: Given correlated subquery is not supported"); + assertThat(assertions.query("SELECT 1 > ALL (VALUES k) FROM (VALUES (1, 1), (1, 2)) t(k, v) GROUP BY k")) + .failure().hasMessageContaining("line 1:17: Given correlated subquery is not supported"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestCheckConstraint.java b/core/trino-main/src/test/java/io/trino/sql/query/TestCheckConstraint.java index 0d63281b4e8fb..fbe09435d839c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestCheckConstraint.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestCheckConstraint.java @@ -34,7 +34,6 @@ import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -164,20 +163,20 @@ public void testInsert() assertThat(assertions.query("INSERT INTO mock.tiny.nation VALUES (101, 'POLAND', 0, 'No comment')")) .matches("SELECT BIGINT '1'"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 11, 'No comment')")) - .hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query("INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 11, 'No comment')")) + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query(""" INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 11, 'No comment'), (27, 'HOLLAND', 11, 'A comment') """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query(""" INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 11, 'No comment'), (27, 'HOLLAND', 11, 'A comment') """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); } /** @@ -205,16 +204,16 @@ WHEN NOT MATCHED THEN INSERT (nationkey) VALUES (0) .matches("SELECT BIGINT '1'"); // Outside allowed check constraint - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 42) t(dummy) ON false WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 10, 'No comment') """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES (26, 'POLAND', 10, 'No comment'), (27, 'HOLLAND', 10, 'A comment')) t(a,b,c,d) ON nationkey = a WHEN NOT MATCHED THEN INSERT VALUES (a,b,c,d) """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); } @Test @@ -233,10 +232,10 @@ public void testInsertCheckMultipleColumns() assertThat(assertions.query("INSERT INTO mock.tiny.nation_multiple_column_constraint VALUES (99, 'POLAND', 49, 'No comment')")) .matches("SELECT BIGINT '1'"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_multiple_column_constraint VALUES (99, 'POLAND', 50, 'No comment')")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_multiple_column_constraint VALUES (100, 'POLAND', 49, 'No comment')")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_multiple_column_constraint VALUES (99, 'POLAND', 50, 'No comment')")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_multiple_column_constraint VALUES (100, 'POLAND', 49, 'No comment')")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); } @Test @@ -245,59 +244,59 @@ public void testInsertSubquery() assertThat(assertions.query("INSERT INTO mock.tiny.nation_subquery VALUES (26, 'POLAND', 51, 'No comment')")) .matches("SELECT BIGINT '1'"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_subquery VALUES (10, 'POLAND', 0, 'No comment')")) - .hasMessage("Check constraint violation: (nationkey > (SELECT count(*)\nFROM\n nation\n))"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_subquery VALUES (10, 'POLAND', 0, 'No comment')")) + .failure().hasMessage("Check constraint violation: (nationkey > (SELECT count(*)\nFROM\n nation\n))"); } @Test public void testInsertUnsupportedCurrentDate() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_current_date VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_current_date VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testInsertUnsupportedCurrentTime() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_current_time VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_current_time VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testInsertUnsupportedCurrentTimestamp() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_current_timestamp VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_current_timestamp VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testInsertUnsupportedLocaltime() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_localtime VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_localtime VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testInsertUnsupportedLocaltimestamp() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_localtimestamp VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_localtimestamp VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testInsertUnsupportedConstraint() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_invalid_function VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Function 'invalid_function' not registered"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_not_boolean_expression VALUES (101, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("to be of type BOOLEAN, but was integer"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_invalid_function VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Function 'invalid_function' not registered"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_not_boolean_expression VALUES (101, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("to be of type BOOLEAN, but was integer"); } @Test public void testInsertNotDeterministic() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_not_deterministic VALUES (100, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should be deterministic"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_not_deterministic VALUES (100, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should be deterministic"); } /** @@ -353,13 +352,13 @@ public void testUpdate() .matches("SELECT BIGINT '3'"); // Outside allowed check constraint - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10")) - .hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10 WHERE nationkey IN (1, 11)")) - .hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10")) + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10 WHERE nationkey IN (1, 11)")) + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10 WHERE nationkey = 11")) - .hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 10 WHERE nationkey = 11")) + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); // Within allowed check constraint, but updated rows are outside the check constraint assertThat(assertions.query("UPDATE mock.tiny.nation SET nationkey = 10 WHERE nationkey < 3")) @@ -394,75 +393,75 @@ public void testUpdateCheckMultipleColumns() .matches("SELECT BIGINT '25'"); // Outside allowed check constraint - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50, nationkey = 100")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50, nationkey = 99")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 49, nationkey = 100")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET nationkey = 100")) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50, nationkey = 100")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50, nationkey = 99")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 49, nationkey = 100")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET regionkey = 50")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + assertThat(assertions.query("UPDATE mock.tiny.nation_multiple_column_constraint SET nationkey = 100")) + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); } @Test public void testUpdateSubquery() { // TODO Support subqueries for UPDATE statement in check constraint - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_subquery SET nationkey = 100")) - .hasMessageContaining("Unexpected subquery expression in logical plan"); + assertThat(assertions.query("UPDATE mock.tiny.nation_subquery SET nationkey = 100")) + .nonTrinoExceptionFailure().hasMessageContaining("Unexpected subquery expression in logical plan"); } @Test public void testUpdateUnsupportedCurrentDate() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_current_date SET nationkey = 10")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("UPDATE mock.tiny.nation_current_date SET nationkey = 10")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testUpdateUnsupportedCurrentTime() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_current_time SET nationkey = 10")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("UPDATE mock.tiny.nation_current_time SET nationkey = 10")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testUpdateUnsupportedCurrentTimestamp() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_current_timestamp SET nationkey = 10")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("UPDATE mock.tiny.nation_current_timestamp SET nationkey = 10")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testUpdateUnsupportedLocaltime() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_localtime SET nationkey = 10")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("UPDATE mock.tiny.nation_localtime SET nationkey = 10")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testUpdateUnsupportedLocaltimestamp() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_localtimestamp SET nationkey = 10")) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + assertThat(assertions.query("UPDATE mock.tiny.nation_localtimestamp SET nationkey = 10")) + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testUpdateUnsupportedConstraint() { - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_invalid_function SET nationkey = 10")) - .hasMessageContaining("Function 'invalid_function' not registered"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_not_boolean_expression SET nationkey = 10")) - .hasMessageContaining("to be of type BOOLEAN, but was integer"); + assertThat(assertions.query("UPDATE mock.tiny.nation_invalid_function SET nationkey = 10")) + .failure().hasMessageContaining("Function 'invalid_function' not registered"); + assertThat(assertions.query("UPDATE mock.tiny.nation_not_boolean_expression SET nationkey = 10")) + .failure().hasMessageContaining("to be of type BOOLEAN, but was integer"); } @Test public void testUpdateNotDeterministic() { - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_not_deterministic VALUES (100, 'POLAND', 0, 'No comment')")) - .hasMessageContaining("Check constraint expression should be deterministic"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_not_deterministic VALUES (100, 'POLAND', 0, 'No comment')")) + .failure().hasMessageContaining("Check constraint expression should be deterministic"); } /** @@ -479,22 +478,22 @@ MERGE INTO mock.tiny.nation USING (VALUES 5) t(x) ON nationkey = x .matches("SELECT BIGINT '1'"); // Merge column within allowed check constraint, but updated rows are outside the check constraint - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 5 """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1, 11) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 5 """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation t USING mock.tiny.nation s ON t.nationkey = s.nationkey WHEN MATCHED THEN UPDATE SET regionkey = 10 """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); // Merge column outside allowed check constraint and updated rows within allowed check constraint assertThat(assertions.query(""" @@ -504,11 +503,11 @@ MERGE INTO mock.tiny.nation USING (VALUES 1, 11) t(x) ON regionkey = x .matches("SELECT BIGINT '5'"); // Merge column outside allowed check constraint and updated rows are outside the check constraint - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 11) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 5 """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); // No check constraining column in query assertThat(assertions.query(""" @@ -546,21 +545,21 @@ WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') .matches("SELECT BIGINT '22'"); // Outside allowed check constraint - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 10 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 9, 'No comment') """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 10, 'No comment') """)) - .hasMessage("Check constraint violation: (regionkey < 10)"); + .failure().hasMessage("Check constraint violation: (regionkey < 10)"); } @Test @@ -576,143 +575,143 @@ WHEN NOT MATCHED THEN INSERT VALUES (99, 'POLAND', 49, 'No comment') .matches("SELECT BIGINT '22'"); // Outside allowed check constraint (regionkey in UPDATE) - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_multiple_column_constraint USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 50 WHEN NOT MATCHED THEN INSERT VALUES (99, 'POLAND', 49, 'No comment') """)) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); // Outside allowed check constraint (regionkey in INSERT) - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_multiple_column_constraint USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 49 WHEN NOT MATCHED THEN INSERT VALUES (99, 'POLAND', 50, 'No comment') """)) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); // Outside allowed check constraint (nationkey in UPDATE) - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_multiple_column_constraint USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET nationkey = 100 WHEN NOT MATCHED THEN INSERT VALUES (99, 'POLAND', 49, 'No comment') """)) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); // Outside allowed check constraint (nationkey in INSERT) - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_multiple_column_constraint USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET nationkey = 99 WHEN NOT MATCHED THEN INSERT VALUES (100, 'POLAND', 50, 'No comment') """)) - .hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); + .failure().hasMessage("Check constraint violation: ((nationkey < 100) AND (regionkey < 50))"); } @Test public void testMergeSubquery() { // TODO https://github.com/trinodb/trino/issues/18230 Support subqueries for MERGE statement in check constraint - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_subquery USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Unexpected subquery expression in logical plan"); + .nonTrinoExceptionFailure().hasMessageContaining("Unexpected subquery expression in logical plan"); } @Test public void testMergeUnsupportedCurrentDate() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_current_date USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testMergeUnsupportedCurrentTime() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_current_time USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testMergeUnsupportedCurrentTimestamp() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_current_timestamp USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testMergeUnsupportedLocaltime() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_localtime USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testMergeUnsupportedLocaltimestamp() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_localtimestamp USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should not contain temporal expression"); + .failure().hasMessageContaining("Check constraint expression should not contain temporal expression"); } @Test public void testMergeUnsupportedConstraint() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_invalid_function USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Function 'invalid_function' not registered"); - assertThatThrownBy(() -> assertions.query(""" + .failure().hasMessageContaining("Function 'invalid_function' not registered"); + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_not_boolean_expression USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("to be of type BOOLEAN, but was integer"); + .failure().hasMessageContaining("to be of type BOOLEAN, but was integer"); } @Test public void testMergeNotDeterministic() { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation_not_deterministic USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED AND t.x = 1 THEN DELETE WHEN MATCHED THEN UPDATE SET regionkey = 9 WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment') """)) - .hasMessageContaining("Check constraint expression should be deterministic"); + .failure().hasMessageContaining("Check constraint expression should be deterministic"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java b/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java index bde51de8b5bb1..290ba1ee76069 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java @@ -51,7 +51,6 @@ import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -533,8 +532,8 @@ public void testRecursion() .expression("(SELECT orderkey FROM orders)") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); // different reference style to same table accessControl.reset(); @@ -549,8 +548,8 @@ public void testRecursion() .expression("(SELECT orderkey FROM local.tiny.orders)") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); // mutual recursion accessControl.reset(); @@ -576,8 +575,8 @@ public void testRecursion() .expression("(SELECT orderkey FROM orders)") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessageMatching(".*\\QColumn mask for 'local.tiny.orders.orderkey' is recursive\\E.*"); } @Test @@ -594,9 +593,9 @@ public void testLimitedScope() .schema("tiny") .expression("orderkey") .build()); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT min(custkey) FROM customer WHERE customer.custkey = orders.custkey) FROM orders")) - .hasMessage("line 1:34: Invalid column mask for 'local.tiny.customer.custkey': Column 'orderkey' cannot be resolved"); + .failure().hasMessage("line 1:34: Invalid column mask for 'local.tiny.customer.custkey': Column 'orderkey' cannot be resolved"); } @Test @@ -635,8 +634,8 @@ public void testInvalidMasks() .expression("$$$") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:22: Invalid column mask for 'local.tiny.orders.orderkey': mismatched input '$'. Expecting: "); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:22: Invalid column mask for 'local.tiny.orders.orderkey': mismatched input '$'. Expecting: "); // unknown column accessControl.reset(); @@ -651,8 +650,8 @@ public void testInvalidMasks() .expression("unknown_column") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:22: Invalid column mask for 'local.tiny.orders.orderkey': Column 'unknown_column' cannot be resolved"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:22: Invalid column mask for 'local.tiny.orders.orderkey': Column 'unknown_column' cannot be resolved"); // invalid type accessControl.reset(); @@ -667,8 +666,8 @@ public void testInvalidMasks() .expression("'foo'") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:22: Expected column mask for 'local.tiny.orders.orderkey' to be of type bigint, but was varchar(3)"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:22: Expected column mask for 'local.tiny.orders.orderkey' to be of type bigint, but was varchar(3)"); // aggregation accessControl.reset(); @@ -683,8 +682,8 @@ public void testInvalidMasks() .expression("count(*) > 0") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:10: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [count(*)]"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:10: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [count(*)]"); // window function accessControl.reset(); @@ -699,8 +698,8 @@ public void testInvalidMasks() .expression("row_number() OVER () > 0") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:22: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [row_number() OVER ()]"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:22: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [row_number() OVER ()]"); // grouping function accessControl.reset(); @@ -715,8 +714,8 @@ public void testInvalidMasks() .expression("grouping(orderkey) = 0") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("line 1:20: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [GROUPING (orderkey)]"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("line 1:20: Column mask for 'orders.orderkey' cannot contain aggregations, window functions or grouping operations: [GROUPING (orderkey)]"); } @Test @@ -785,8 +784,8 @@ public void testColumnMaskingUsingRestrictedColumn() .identity(USER) .expression("custkey") .build()); - assertThatThrownBy(() -> assertions.query("SELECT orderkey FROM orders")) - .hasMessage("Access Denied: Cannot select from columns [orderkey, custkey] in table or view local.tiny.orders"); + assertThat(assertions.query("SELECT orderkey FROM orders")) + .failure().hasMessage("Access Denied: Cannot select from columns [orderkey, custkey] in table or view local.tiny.orders"); } @Test @@ -801,8 +800,8 @@ public void testInsertWithColumnMasking() .identity(USER) .expression("clerk") .build()); - assertThatThrownBy(() -> assertions.query("INSERT INTO orders SELECT * FROM orders")) - .hasMessage("Insert into table with column masks is not supported"); + assertThat(assertions.query("INSERT INTO orders SELECT * FROM orders")) + .failure().hasMessage("Insert into table with column masks is not supported"); } @Test @@ -817,8 +816,8 @@ public void testDeleteWithColumnMasking() .identity(USER) .expression("clerk") .build()); - assertThatThrownBy(() -> assertions.query("DELETE FROM orders")) - .hasMessage("line 1:1: Delete from table with column mask"); + assertThat(assertions.query("DELETE FROM orders")) + .failure().hasMessage("line 1:1: Delete from table with column mask"); } @Test @@ -833,12 +832,12 @@ public void testUpdateWithColumnMasking() .identity(USER) .expression("clerk") .build()); - assertThatThrownBy(() -> assertions.query("UPDATE orders SET clerk = 'X'")) - .hasMessage("line 1:1: Updating a table with column masks is not supported"); - assertThatThrownBy(() -> assertions.query("UPDATE orders SET orderkey = -orderkey")) - .hasMessage("line 1:1: Updating a table with column masks is not supported"); - assertThatThrownBy(() -> assertions.query("UPDATE orders SET clerk = 'X', orderkey = -orderkey")) - .hasMessage("line 1:1: Updating a table with column masks is not supported"); + assertThat(assertions.query("UPDATE orders SET clerk = 'X'")) + .failure().hasMessage("line 1:1: Updating a table with column masks is not supported"); + assertThat(assertions.query("UPDATE orders SET orderkey = -orderkey")) + .failure().hasMessage("line 1:1: Updating a table with column masks is not supported"); + assertThat(assertions.query("UPDATE orders SET clerk = 'X', orderkey = -orderkey")) + .failure().hasMessage("line 1:1: Updating a table with column masks is not supported"); } @Test @@ -906,12 +905,12 @@ public void testColumnMaskWithHiddenColumns() .assertThat() .skippingTypesCheck() .matches("VALUES 'POLAND'"); - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column SELECT * FROM mock.tiny.nation_with_hidden_column")) - .hasMessage("Insert into table with column masks is not supported"); - assertThatThrownBy(() -> assertions.query("DELETE FROM mock.tiny.nation_with_hidden_column")) - .hasMessage("line 1:1: Delete from table with column mask"); - assertThatThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'X'")) - .hasMessage("line 1:1: Updating a table with column masks is not supported"); + assertThat(assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column SELECT * FROM mock.tiny.nation_with_hidden_column")) + .failure().hasMessage("Insert into table with column masks is not supported"); + assertThat(assertions.query("DELETE FROM mock.tiny.nation_with_hidden_column")) + .failure().hasMessage("line 1:1: Delete from table with column mask"); + assertThat(assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'X'")) + .failure().hasMessage("line 1:1: Updating a table with column masks is not supported"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestDistinctWithOrderBy.java b/core/trino-main/src/test/java/io/trino/sql/query/TestDistinctWithOrderBy.java index 02023a2dd4253..68b7447f8fddf 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestDistinctWithOrderBy.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestDistinctWithOrderBy.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -85,13 +84,13 @@ public void testSelectAllAliases() @Test public void testColumnAliasing() { - assertThatThrownBy(() -> assertions.query("SELECT DISTINCT 1 AS a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) - .hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); + assertThat(assertions.query("SELECT DISTINCT 1 AS a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) + .failure().hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); - assertThatThrownBy(() -> assertions.query("SELECT DISTINCT -a AS a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) - .hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); + assertThat(assertions.query("SELECT DISTINCT -a AS a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) + .failure().hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); - assertThatThrownBy(() -> assertions.query("SELECT DISTINCT a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) - .hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); + assertThat(assertions.query("SELECT DISTINCT a, a + b FROM (VALUES (1, 2)) t(a, b) ORDER BY a + b")) + .failure().hasMessage("line 1:1: For SELECT DISTINCT, ORDER BY expressions must appear in select list"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestExecuteImmediate.java b/core/trino-main/src/test/java/io/trino/sql/query/TestExecuteImmediate.java index 378e26bb72b0e..f647782c1ed31 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestExecuteImmediate.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestExecuteImmediate.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static io.trino.spi.StandardErrorCode.SYNTAX_ERROR; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -60,10 +59,12 @@ public void testQuotesInStatement() @Test public void testSyntaxError() { - assertTrinoExceptionThrownBy(() -> assertions.query("EXECUTE IMMEDIATE 'SELECT ''foo'")) + assertThat(assertions.query("EXECUTE IMMEDIATE 'SELECT ''foo'")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessageMatching("line 1:27: mismatched input '''. Expecting: .*"); - assertTrinoExceptionThrownBy(() -> assertions.query("EXECUTE IMMEDIATE\n'SELECT ''foo'")) + assertThat(assertions.query("EXECUTE IMMEDIATE\n'SELECT ''foo'")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessageMatching("line 2:8: mismatched input '''. Expecting: .*"); } @@ -71,9 +72,11 @@ public void testSyntaxError() @Test public void testSemanticError() { - assertTrinoExceptionThrownBy(() -> assertions.query("EXECUTE IMMEDIATE 'SELECT * FROM tiny.tpch.orders'")) + assertThat(assertions.query("EXECUTE IMMEDIATE 'SELECT * FROM tiny.tpch.orders'")) + .failure() .hasMessageMatching("line 1:34: Catalog 'tiny' not found"); - assertTrinoExceptionThrownBy(() -> assertions.query("EXECUTE IMMEDIATE\n'SELECT *\nFROM tiny.tpch.orders'")) + assertThat(assertions.query("EXECUTE IMMEDIATE\n'SELECT *\nFROM tiny.tpch.orders'")) + .failure() .hasMessageMatching("line 3:6: Catalog 'tiny' not found"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestExpressions.java b/core/trino-main/src/test/java/io/trino/sql/query/TestExpressions.java index d7892e1a48a1d..1c43855bee7d6 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestExpressions.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestExpressions.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -65,8 +64,8 @@ public void testInShortCircuit() assertThat(assertions.query("SELECT IF(3 IN (2, 4, 3, 5 / 0), 1e0, x + x) FROM (VALUES rand()) t(x)")).matches("VALUES 1e0"); // the in-predicate is inlined into Values and evaluated by the ExpressionInterpreter: eager evaluation, failure. - assertThatThrownBy(() -> assertions.query("SELECT 3 IN (2, 4, 3, 5 / 0)")) - .hasMessage("Division by zero"); + assertThat(assertions.query("SELECT 3 IN (2, 4, 3, 5 / 0)")) + .failure().hasMessage("Division by zero"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestFilterInaccessibleColumns.java b/core/trino-main/src/test/java/io/trino/sql/query/TestFilterInaccessibleColumns.java index b0f6c3db6181d..e8237332a458a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestFilterInaccessibleColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestFilterInaccessibleColumns.java @@ -33,7 +33,6 @@ import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -95,7 +94,7 @@ public void testDescribeBaseline() accessControl.reset(); assertThat(assertions.query("DESCRIBE nation")) - .matches(materializedRows -> materializedRows + .result().matches(materializedRows -> materializedRows .getMaterializedRows().stream() .anyMatch(materializedRow -> materializedRow.getField(0).equals("comment"))); } @@ -107,7 +106,7 @@ public void testDescribe() accessControl.deny(privilege(USER, "nation.comment", SELECT_COLUMN)); assertThat(assertions.query("DESCRIBE nation")) - .matches(materializedRows -> materializedRows + .result().matches(materializedRows -> materializedRows .getMaterializedRows().stream() .noneMatch(materializedRow -> materializedRow.getField(0).equals("comment"))); } @@ -118,7 +117,7 @@ public void testShowColumnsBaseline() accessControl.reset(); assertThat(assertions.query("SHOW COLUMNS FROM nation")) - .matches(materializedRows -> materializedRows + .result().matches(materializedRows -> materializedRows .getMaterializedRows().stream() .anyMatch(materializedRow -> materializedRow.getField(0).equals("comment"))); } @@ -130,7 +129,7 @@ public void testShowColumns() accessControl.deny(privilege("nation.comment", SELECT_COLUMN)); assertThat(assertions.query("SHOW COLUMNS FROM nation")) - .matches(materializedRows -> materializedRows + .result().matches(materializedRows -> materializedRows .getMaterializedRows().stream() .noneMatch(materializedRow -> materializedRow.getField(0).equals("comment"))); } @@ -149,8 +148,8 @@ public void testFilterExplicitSelect() .matches("VALUES (BIGINT '6', CAST('FRANCE' AS VARCHAR(25)), BIGINT '3')"); // Select all columns explicitly - assertThatThrownBy(() -> assertions.query("SELECT nationkey, name, regionkey, comment FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query("SELECT nationkey, name, regionkey, comment FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); } @Test @@ -185,8 +184,8 @@ public void testRowFilterWithoutAccessToInaccessibleColumn() .expression("comment IS NOT null") .build()); accessControl.deny(privilege(USER, "nation.comment", SELECT_COLUMN)); - assertThatThrownBy(() -> assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); } @Test @@ -204,8 +203,8 @@ public void testRowFilterAsSessionUserOnInaccessibleColumn() accessControl.rowFilter(table, ADMIN, filter); accessControl.rowFilter(table, USER, filter); - assertThatThrownBy(() -> assertions.query(user(USER), "SELECT * FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query(user(USER), "SELECT * FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); assertThat(assertions.query(user(ADMIN), "SELECT * FROM nation WHERE name = 'FRANCE'")) .matches("VALUES (BIGINT '6', CAST('FRANCE' AS VARCHAR(25)), BIGINT '3', CAST('refully final requests. regular, ironi' AS VARCHAR(152)))"); } @@ -244,8 +243,8 @@ public void testMaskingWithoutAccessToInaccessibleColumn() .expression("CASE nationkey WHEN 6 THEN 'masked-comment' ELSE comment END") .build()); - assertThatThrownBy(() -> assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); } @Test @@ -286,8 +285,8 @@ public void testMaskingAsSessionUserWithCaseOnInaccessibleColumn() accessControl.columnMask(table, "comment", ADMIN, mask); accessControl.columnMask(table, "comment", USER, mask); - assertThatThrownBy(() -> assertions.query(user(USER), "SELECT * FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query(user(USER), "SELECT * FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); assertThat(assertions.query(user(ADMIN), "SELECT * FROM nation WHERE name = 'CANADA'")) .matches("VALUES (BIGINT '3', CAST('CANADA' AS VARCHAR(25)), BIGINT '1', CAST('masked-comment' AS VARCHAR(152)))"); } @@ -299,8 +298,8 @@ public void testPredicateOnInaccessibleColumn() // Hide name but use it in the query predicate accessControl.deny(privilege(USER, "nation.name", SELECT_COLUMN)); - assertThatThrownBy(() -> assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) - .hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query("SELECT * FROM nation WHERE name = 'FRANCE'")) + .failure().hasMessage("Access Denied: Cannot select from columns [nationkey, regionkey, name, comment] in table or view test_catalog.tiny.nation"); } @Test @@ -309,8 +308,8 @@ public void testJoinBaseline() accessControl.reset(); assertThat(assertions.query("SELECT * FROM nation,customer WHERE customer.nationkey = nation.nationkey AND nation.name = 'FRANCE' AND customer.name='Customer#000001477'")) - .matches(materializedRows -> - materializedRows.getMaterializedRows().get(0).getField(11).equals("ites nag blithely alongside of the ironic accounts. accounts use. carefully silent deposits")); + .result().matches(materializedRows -> + materializedRows.getMaterializedRows().get(0).getField(11).equals("ites nag blithely alongside of the ironic accounts. accounts use. carefully silent deposits")); } @Test @@ -320,7 +319,7 @@ public void testJoin() accessControl.deny(privilege(USER, "nation.comment", SELECT_COLUMN)); assertThat(assertions.query("SELECT * FROM nation,customer WHERE customer.nationkey = nation.nationkey AND nation.name = 'FRANCE' AND customer.name='Customer#000001477'")) - .matches(materializedRows -> + .result().matches(materializedRows -> materializedRows.getMaterializedRows().get(0).getFields().size() == 11); } @@ -348,8 +347,8 @@ public void testFunctionOnInaccessibleColumn() accessControl.reset(); accessControl.deny(privilege(USER, "nation.name", SELECT_COLUMN)); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (SELECT concat(name,'-test') FROM nation WHERE name = 'FRANCE')")) - .hasMessage("Access Denied: Cannot select from columns [name] in table or view test_catalog.tiny.nation"); + assertThat(assertions.query("SELECT * FROM (SELECT concat(name,'-test') FROM nation WHERE name = 'FRANCE')")) + .failure().hasMessage("Access Denied: Cannot select from columns [name] in table or view test_catalog.tiny.nation"); } private Session user(String user) diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestFormat.java b/core/trino-main/src/test/java/io/trino/sql/query/TestFormat.java index 44e141d30992a..962514e6791c3 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestFormat.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestFormat.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -49,7 +48,7 @@ public void testAggregationInFormat() assertThat(assertions.query("SELECT format('%s', sum(k)) FROM (VALUES 1, 2, 3) t(k)")).matches("VALUES VARCHAR '6'"); assertThat(assertions.query("SELECT format(arbitrary(s), sum(k)) FROM (VALUES ('%s', 1), ('%s', 2), ('%s', 3)) t(s, k)")).matches("VALUES VARCHAR '6'"); - assertThatThrownBy(() -> assertions.query("SELECT format(s, 1) FROM (VALUES ('%s', 1)) t(s, k) GROUP BY k")) - .hasMessageMatching("\\Qline 1:8: 'format(s, 1)' must be an aggregate expression or appear in GROUP BY clause\\E"); + assertThat(assertions.query("SELECT format(s, 1) FROM (VALUES ('%s', 1)) t(s, k) GROUP BY k")) + .failure().hasMessageMatching("\\Qline 1:8: 'format(s, 1)' must be an aggregate expression or appear in GROUP BY clause\\E"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestFullJoin.java b/core/trino-main/src/test/java/io/trino/sql/query/TestFullJoin.java index fbe73bff2079e..bc7a2b11204a0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestFullJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestFullJoin.java @@ -39,9 +39,9 @@ public void testFullJoinWithLimit() { assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2) AS l(v) FULL OUTER JOIN (VALUES 2, 1) AS r(v) ON l.v = r.v LIMIT 1")) - .satisfies(actual -> assertThat(actual.getMaterializedRows()) - .hasSize(1) - .containsAnyElementsOf(assertions.execute("VALUES (1,1), (2,2)").getMaterializedRows())); + .result().rows() + .hasSize(1) + .containsAnyElementsOf(assertions.execute("VALUES (1,1), (2,2)").getMaterializedRows()); assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2) AS l(v) FULL OUTER JOIN (VALUES 2) AS r(v) ON l.v = r.v " + diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestGroupBy.java b/core/trino-main/src/test/java/io/trino/sql/query/TestGroupBy.java index c3f02573f71dc..3cfe153ecf0e8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestGroupBy.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestGroupBy.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -67,11 +66,11 @@ public void testCastDifferentCase() .matches("SELECT CAST(row(BIGINT '42') AS row(\"A\" bigint))"); // ROW field name in a different case, delimited - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT CAST(row(x) AS row(\"a\" bigint)) " + "FROM (VALUES 42) t(x) " + "GROUP BY CAST(row(x) AS row(\"A\" bigint))")) - .hasMessage("line 1:8: 'CAST(ROW (x) AS ROW(\"a\" bigint))' must be an aggregate expression or appear in GROUP BY clause"); + .failure().hasMessage("line 1:8: 'CAST(ROW (x) AS ROW(\"a\" bigint))' must be an aggregate expression or appear in GROUP BY clause"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJoin.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJoin.java index 2e07a02974c48..f6b371ed8dee7 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJoin.java @@ -33,7 +33,6 @@ import static io.trino.sql.planner.plan.JoinNode.Type.INNER; import static java.util.function.Predicate.not; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -185,53 +184,53 @@ public void testInPredicateInJoinCriteria() .matches("VALUES (1,1)"); // correlation in join clause not allowed for outer join - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) FULL JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES t.x)")) - .hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) FULL JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES u.x)")) - .hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) FULL JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES t.x)")) - .hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) FULL JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES u.x)")) - .hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) LEFT JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES t.x)")) - .hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) LEFT JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES u.x)")) - .hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) LEFT JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES t.x)")) - .hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) LEFT JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES u.x)")) - .hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:93: Reference to column 'u.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES t.x)")) - .hasMessage("line 1:94: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:94: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON u.x IN (VALUES u.x)")) - .hasMessage("line 1:94: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:94: Reference to column 'u.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES t.x)")) - .hasMessage("line 1:94: Reference to column 't.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:94: Reference to column 't.x' from outer scope not allowed in this context"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON t.x IN (VALUES u.x)")) - .hasMessage("line 1:94: Reference to column 'u.x' from outer scope not allowed in this context"); + .failure().hasMessage("line 1:94: Reference to column 'u.x' from outer scope not allowed in this context"); } @Test @@ -247,7 +246,8 @@ public void testQuantifiedComparisonInJoinCriteria() // StatementAnalyzer.visitJoin needs to be updated to check whether the join criteria is an InPredicate or QualifiedComparison // with mixed references to both sides of the join. For that, the Expression needs to be analyzed against a hybrid scope made of both branches // of the join, instead of using the output scope of the Join node. This, in turn requires adding support for multiple scopes in ExpressionAnalyzer - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON t.x + u.x > ALL (VALUES 1)")); + assertThat(assertions.query("SELECT * FROM (VALUES 1, 2, NULL) t(x) RIGHT JOIN (VALUES 1, 3, NULL) u(x) ON t.x + u.x > ALL (VALUES 1)")) + .nonTrinoExceptionFailure().hasMessageMatching("Invalid node. Expression dependencies .* not in source plan output .*"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJoinUsing.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJoinUsing.java index 44c5413419977..01ccb309dca07 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJoinUsing.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJoinUsing.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -44,11 +43,11 @@ public void testColumnReferences() "(VALUES (1, 'b')) AS u(k, v2) USING (k)")) .matches("VALUES (1, 'a', 'b', 'a', 'b')"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT t.k FROM " + "(VALUES (1, 'a')) AS t(k, v1) JOIN" + "(VALUES (1, 'b')) AS u(k, v2) USING (k)")) - .hasMessageMatching(".*Column 't.k' cannot be resolved.*"); + .failure().hasMessageMatching(".*Column 't.k' cannot be resolved.*"); } @Test @@ -117,11 +116,11 @@ public void testCoercion() @Test public void testDuplicateColumns() { - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM " + "(VALUES (1, 'a')) AS t(k, v1) JOIN" + "(VALUES (1, 'b')) AS u(k, v2) USING (k, k)")) - .hasMessageMatching(".*Column 'k' appears multiple times in USING clause.*"); + .failure().hasMessageMatching(".*Column 'k' appears multiple times in USING clause.*"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonArrayFunction.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonArrayFunction.java index cd2650fad0151..687a0cba77e86 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonArrayFunction.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonArrayFunction.java @@ -24,7 +24,6 @@ import static com.google.common.io.BaseEncoding.base16; import static io.trino.spi.StandardErrorCode.JSON_INPUT_CONVERSION_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; @@ -100,8 +99,9 @@ public void testElementWithFormat() .matches("VALUES VARCHAR '[{\"a\":1}]'"); // malformed string to be read as JSON - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_array('[...' FORMAT JSON)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR); // duplicate key inside the formatted element: only one entry is retained @@ -128,8 +128,9 @@ public void testElementTypes() .matches("VALUES VARCHAR '[\"2001-01-31\"]'"); // HyperLogLog cannot be cast to varchar - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_array(approx_set(1))")) + .failure() .hasErrorCode(NOT_SUPPORTED); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonExistsFunction.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonExistsFunction.java index 5100e263f13d2..786b00852ae50 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonExistsFunction.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonExistsFunction.java @@ -26,7 +26,6 @@ import static io.trino.spi.StandardErrorCode.PATH_EVALUATION_ERROR; import static io.trino.spi.StandardErrorCode.SYNTAX_ERROR; import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; @@ -82,8 +81,9 @@ public void testJsonExists() "SELECT json_exists('" + INPUT + "', 'strict $[100]' UNKNOWN ON ERROR)")) .matches("VALUES cast(null AS boolean)"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INPUT + "', 'strict $[100]' ERROR ON ERROR)")) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript: [100, 100] for array of size 3"); } @@ -101,8 +101,9 @@ public void testInputFormat() "SELECT json_exists('" + INPUT + "' FORMAT JSON, 'lax $[1]')")) .matches("VALUES true"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INPUT + "' FORMAT JSON ENCODING UTF8, 'lax $[1]')")) + .failure() .hasErrorCode(TYPE_MISMATCH) .hasMessage("line 1:20: Cannot read input of type varchar(15) as JSON using formatting JSON ENCODING UTF8"); @@ -138,9 +139,9 @@ public void testInputFormat() .matches("VALUES true"); // the encoding must match the actual data - String finalVarbinaryLiteral = varbinaryLiteral; - assertTrinoExceptionThrownBy(() -> assertions.query( - "SELECT json_exists(" + finalVarbinaryLiteral + " FORMAT JSON ENCODING UTF8, 'lax $[1]' ERROR ON ERROR)")) + assertThat(assertions.query( + "SELECT json_exists(" + varbinaryLiteral + " FORMAT JSON ENCODING UTF8, 'lax $[1]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); } @@ -167,8 +168,9 @@ public void testInputConversionError() "SELECT json_exists('" + INCORRECT_INPUT + "', 'strict $[1]' UNKNOWN ON ERROR)")) .matches("VALUES cast(null AS boolean)"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INCORRECT_INPUT + "', 'strict $[1]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); } @@ -177,8 +179,9 @@ public void testInputConversionError() public void testPassingClause() { // watch out for case sensitive identifiers in JSON path - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INPUT + "', 'lax $number + 1' PASSING 2 AS number)")) + .failure() .hasErrorCode(INVALID_PATH) .hasMessage("line 1:39: no value passed for parameter number. Try quoting \"number\" in the PASSING clause to match case"); @@ -196,8 +199,9 @@ public void testPassingClause() "SELECT json_exists('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\")")) .matches("VALUES false"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\" ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); @@ -210,8 +214,9 @@ public void testPassingClause() @Test public void testIncorrectPath() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_exists('" + INPUT + "', 'certainly not a valid path')")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:40: mismatched input 'certainly' expecting {'lax', 'strict'}"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonObjectFunction.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonObjectFunction.java index fa484e56e970f..ee332a358df04 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonObjectFunction.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonObjectFunction.java @@ -25,7 +25,6 @@ import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.StandardErrorCode.JSON_INPUT_CONVERSION_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; @@ -80,8 +79,9 @@ public void testMultipleMembers() @Test public void testNullKey() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object(CAST(null AS varchar) : 1)")) + .failure() .hasErrorCode(INVALID_FUNCTION_ARGUMENT) .hasMessage("null value passed for JSON object key to JSON_OBJECT function"); } @@ -109,19 +109,22 @@ public void testNullValue() public void testDuplicateKey() { // we don't support it because it requires creating a JSON object with duplicate key - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : 1, 'key' : 2 WITHOUT UNIQUE KEYS)")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("cannot construct a JSON object with duplicate key"); // WITHOUT UNIQUE KEYS is the default option - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : 1, 'key' : 2)")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("cannot construct a JSON object with duplicate key"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : 1, 'key' : 2 WITH UNIQUE KEYS)")) + .failure() .hasErrorCode(INVALID_FUNCTION_ARGUMENT) .hasMessage("duplicate key passed to JSON_OBJECT function"); } @@ -142,8 +145,9 @@ public void testValueWithFormat() .matches("VALUES VARCHAR '{\"key\":{\"a\":1}}'"); // malformed string to be read as JSON - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : '[...' FORMAT JSON)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR); // duplicate key inside the formatted value: only one entry is retained @@ -156,8 +160,9 @@ public void testValueWithFormat() .matches("VALUES VARCHAR '{\"key\":{\"a\":1}}'"); // in presence of input value with FORMAT, the option WITH UNIQUE KEYS is not supported, because the input function does not support this semantics - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : '{\"a\" : 1, \"a\" : 1}' FORMAT JSON WITH UNIQUE KEYS)")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:8: WITH UNIQUE KEYS behavior is not supported for JSON_OBJECT function when input expression has FORMAT"); } @@ -180,8 +185,9 @@ public void testValueTypes() .matches("VALUES VARCHAR '{\"key\":\"2001-01-31\"}'"); // HyperLogLog cannot be cast to varchar - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_object('key' : (approx_set(1)))")) + .failure() .hasErrorCode(NOT_SUPPORTED); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonQueryFunction.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonQueryFunction.java index 51912a88a6b77..562330e3354cc 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonQueryFunction.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonQueryFunction.java @@ -25,11 +25,9 @@ import static io.trino.spi.StandardErrorCode.JSON_OUTPUT_CONVERSION_ERROR; import static io.trino.spi.StandardErrorCode.PATH_EVALUATION_ERROR; import static io.trino.spi.StandardErrorCode.SYNTAX_ERROR; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -78,8 +76,9 @@ public void testJsonQuery() "SELECT json_query('" + INPUT + "', 'strict $[100]' EMPTY OBJECT ON ERROR)")) .matches("VALUES VARCHAR '{}'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'strict $[100]' ERROR ON ERROR)")) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript: [100, 100] for array of size 3"); @@ -102,8 +101,9 @@ public void testJsonQuery() "SELECT json_query('" + INPUT + "', 'lax $[100]' EMPTY OBJECT ON EMPTY)")) .matches("VALUES VARCHAR '{}'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax $[100]' ERROR ON EMPTY)")) + .failure() .hasErrorCode(JSON_OUTPUT_CONVERSION_ERROR) .hasMessage("conversion from JSON failed: JSON path found no items"); @@ -126,8 +126,9 @@ public void testJsonQuery() "SELECT json_query('" + INPUT + "', 'lax $[0 to 2]' EMPTY OBJECT ON ERROR)")) .matches("VALUES VARCHAR '{}'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax $[0 to 2]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_OUTPUT_CONVERSION_ERROR) .hasMessage("conversion from JSON failed: JSON path found multiple items"); } @@ -145,9 +146,9 @@ public void testInputFormat() "SELECT json_query('" + INPUT + "' FORMAT JSON, 'lax $[1]')")) .matches("VALUES VARCHAR '\"b\"'"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "' FORMAT JSON ENCODING UTF8, 'lax $[1]')")) - .hasMessage("line 1:19: Cannot read input of type varchar(15) as JSON using formatting JSON ENCODING UTF8"); + .failure().hasMessage("line 1:19: Cannot read input of type varchar(15) as JSON using formatting JSON ENCODING UTF8"); // FORMAT JSON is default for binary string input byte[] bytes = INPUT.getBytes(UTF_8); @@ -182,9 +183,9 @@ public void testInputFormat() // the encoding must match the actual data String finalVarbinaryLiteral = varbinaryLiteral; - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query(" + finalVarbinaryLiteral + " FORMAT JSON ENCODING UTF8, 'lax $[1]' ERROR ON ERROR)")) - .hasMessage("conversion to JSON failed: "); + .failure().hasMessage("conversion to JSON failed: "); } @Test @@ -209,8 +210,9 @@ public void testInputConversionError() "SELECT json_query('" + INCORRECT_INPUT + "', 'lax $[1]' EMPTY OBJECT ON ERROR)")) .matches("VALUES VARCHAR '{}'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INCORRECT_INPUT + "', 'lax $[1]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); } @@ -219,9 +221,9 @@ public void testInputConversionError() public void testPassingClause() { // watch out for case sensitive identifiers in JSON path - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax $number + 1' PASSING 2 AS number)")) - .hasMessage("line 1:38: no value passed for parameter number. Try quoting \"number\" in the PASSING clause to match case"); + .failure().hasMessage("line 1:38: no value passed for parameter number. Try quoting \"number\" in the PASSING clause to match case"); assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax $number + 1' PASSING 5 AS \"number\")")) @@ -237,8 +239,9 @@ public void testPassingClause() "SELECT json_query('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\")")) .matches("VALUES cast(null AS varchar)"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\" ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); @@ -289,9 +292,9 @@ public void testOutput() .matches("VALUES cast('\"text too ' AS char(10))"); // invalid returned type - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'lax 1' RETURNING tinyint)")) - .hasMessage("line 1:8: Cannot output JSON value as tinyint using formatting JSON"); + .failure().hasMessage("line 1:8: Cannot output JSON value as tinyint using formatting JSON"); // returned type varbinary @@ -395,8 +398,9 @@ public void testQuotesBehavior() @Test public void testIncorrectPath() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_query('" + INPUT + "', 'certainly not a valid path')")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:39: mismatched input 'certainly' expecting {'lax', 'strict'}"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java index c5f15ed662057..a1aecaf8ecde2 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java @@ -21,7 +21,6 @@ import static com.google.common.io.BaseEncoding.base16; import static io.trino.spi.StandardErrorCode.PATH_EVALUATION_ERROR; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -480,7 +479,7 @@ FROM JSON_TABLE( .returnsEmptyResult(); // error during root path evaluation handled according to top level ERROR ON ERROR clause - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM JSON_TABLE( '[]', @@ -488,6 +487,7 @@ FROM JSON_TABLE( COLUMNS(a integer PATH 'lax 1') ERROR ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); } @@ -525,7 +525,7 @@ PLAN DEFAULT(INNER)) .returnsEmptyResult(); // error during nested path evaluation handled according to top level ERROR ON ERROR clause - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM JSON_TABLE( '[]', @@ -537,6 +537,7 @@ FROM JSON_TABLE( PLAN DEFAULT(INNER) ERROR ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); } @@ -545,7 +546,7 @@ PLAN DEFAULT(INNER) public void testColumnPathErrorHandling() { // error during column path evaluation handled according to column's ERROR ON ERROR clause - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM JSON_TABLE( '[]', @@ -553,6 +554,7 @@ FROM JSON_TABLE( COLUMNS(a integer PATH 'strict $[42]' ERROR ON ERROR) EMPTY ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); @@ -568,7 +570,7 @@ FROM JSON_TABLE( .matches("VALUES CAST(null as integer)"); // error during column path evaluation handled according to column's ON ERROR clause which defaults to ERROR ON ERROR because the top level error behavior is ERROR ON ERROR - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM JSON_TABLE( '[]', @@ -576,6 +578,7 @@ FROM JSON_TABLE( COLUMNS(a integer PATH 'strict $[42]') ERROR ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); } @@ -644,7 +647,7 @@ FROM JSON_TABLE( public void testNullPathParameter() { // null as SQL-value parameter "index" is evaluated to a JSON null, and causes type mismatch - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM (SELECT '[1, 2, 3]', CAST(null AS integer)) t(json_col, index_col), JSON_TABLE( @@ -653,11 +656,12 @@ public void testNullPathParameter() COLUMNS(a integer PATH 'lax 1') ERROR ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: invalid item type. Expected: NUMBER, actual: NULL"); // null as JSON (formatted) parameter "index" evaluates to empty sequence, and causes type mismatch - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT * FROM (SELECT '[1, 2, 3]', CAST(null AS varchar)) t(json_col, index_col), JSON_TABLE( @@ -666,6 +670,7 @@ public void testNullPathParameter() COLUMNS(a integer PATH 'lax 1') ERROR ON ERROR) """)) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: array subscript 'from' value must be singleton numeric"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonValueFunction.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonValueFunction.java index e7c5b734715e7..e3132cc5ca692 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonValueFunction.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonValueFunction.java @@ -26,7 +26,6 @@ import static io.trino.spi.StandardErrorCode.JSON_VALUE_RESULT_ERROR; import static io.trino.spi.StandardErrorCode.PATH_EVALUATION_ERROR; import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.nio.charset.StandardCharsets.UTF_16LE; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; @@ -78,8 +77,9 @@ public void testJsonValue() "SELECT json_value('" + INPUT + "', 'strict $[100]' DEFAULT 'x' ON ERROR)")) .matches("VALUES VARCHAR 'x'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'strict $[100]' ERROR ON ERROR)")) + .failure() .hasErrorCode(PATH_EVALUATION_ERROR) .hasMessage("path evaluation failed: structural error: invalid array subscript: [100, 100] for array of size 3"); @@ -98,8 +98,9 @@ public void testJsonValue() "SELECT json_value('" + INPUT + "', 'lax $[100]' DEFAULT 'x' ON EMPTY)")) .matches("VALUES VARCHAR 'x'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'lax $[100]' ERROR ON EMPTY)")) + .failure() .hasErrorCode(JSON_VALUE_RESULT_ERROR) .hasMessage("cannot extract SQL scalar from JSON: JSON path found no items"); @@ -118,8 +119,9 @@ public void testJsonValue() "SELECT json_value('" + INPUT + "', 'lax $[0 to 2]' DEFAULT 'x' ON ERROR)")) .matches("VALUES VARCHAR 'x'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'lax $[0 to 2]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_VALUE_RESULT_ERROR) .hasMessage("cannot extract SQL scalar from JSON: JSON path found multiple items"); } @@ -137,8 +139,9 @@ public void testInputFormat() "SELECT json_value('" + INPUT + "' FORMAT JSON, 'lax $[1]')")) .matches("VALUES VARCHAR 'b'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "' FORMAT JSON ENCODING UTF8, 'lax $[1]')")) + .failure() .hasErrorCode(TYPE_MISMATCH) .hasMessage("line 1:19: Cannot read input of type varchar(15) as JSON using formatting JSON ENCODING UTF8"); @@ -175,8 +178,9 @@ public void testInputFormat() // the encoding must match the actual data String finalVarbinaryLiteral = varbinaryLiteral; - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value(" + finalVarbinaryLiteral + " FORMAT JSON ENCODING UTF8, 'lax $[1]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); } @@ -199,8 +203,9 @@ public void testInputConversionError() "SELECT json_value('" + INCORRECT_INPUT + "', 'lax $[1]' DEFAULT 'x' ON ERROR)")) .matches("VALUES VARCHAR 'x'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INCORRECT_INPUT + "', 'lax $[1]' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); } @@ -209,8 +214,9 @@ public void testInputConversionError() public void testPassingClause() { // watch out for case sensitive identifiers in JSON path - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'lax $number + 1' PASSING 2 AS number)")) + .failure() .hasErrorCode(INVALID_PATH) .hasMessage("line 1:38: no value passed for parameter number. Try quoting \"number\" in the PASSING clause to match case"); @@ -228,8 +234,9 @@ public void testPassingClause() "SELECT json_value('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\")")) .matches("VALUES cast(null AS varchar)"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'lax $array[0]' PASSING '[...' FORMAT JSON AS \"array\" ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_INPUT_CONVERSION_ERROR) .hasMessage("conversion to JSON failed: "); @@ -302,8 +309,9 @@ public void testPathResultNonScalar() "SELECT json_value('" + INPUT + "', 'lax $' DEFAULT 'x' ON ERROR)")) .matches("VALUES VARCHAR 'x'"); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'lax $' ERROR ON ERROR)")) + .failure() .hasErrorCode(JSON_VALUE_RESULT_ERROR) .hasMessage("cannot extract SQL scalar from JSON: JSON path found an item that cannot be converted to an SQL value"); } @@ -311,8 +319,9 @@ public void testPathResultNonScalar() @Test public void testIncorrectPath() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT json_value('" + INPUT + "', 'certainly not a valid path')")) + .failure() .hasMessage("line 1:39: mismatched input 'certainly' expecting {'lax', 'strict'}"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestLag.java b/core/trino-main/src/test/java/io/trino/sql/query/TestLag.java index 58c5b983ce899..33282cfdbcbdd 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestLag.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestLag.java @@ -15,7 +15,7 @@ import org.junit.jupiter.api.Test; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThat; public class TestLag { @@ -23,11 +23,11 @@ public class TestLag public void testNullOffset() { try (QueryAssertions assertions = new QueryAssertions()) { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT lag(v, null) OVER (ORDER BY k) FROM (VALUES (1, 10), (2, 20)) t(k, v) """)) - .hasMessageMatching("Offset must not be null"); + .failure().hasMessageMatching("Offset must not be null"); } } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestLateral.java b/core/trino-main/src/test/java/io/trino/sql/query/TestLateral.java index 5fd23319c24dc..f4ef2e300bf8d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestLateral.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestLateral.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -68,7 +67,7 @@ public void testUncorrelatedLateral() @Test public void testNotInScope() { - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 1) t(a), (SELECT * FROM LATERAL (SELECT a))")) - .hasMessage("line 1:63: Column 'a' cannot be resolved"); + assertThat(assertions.query("SELECT * FROM (VALUES 1) t(a), (SELECT * FROM LATERAL (SELECT a))")) + .failure().hasMessage("line 1:63: Column 'a' cannot be resolved"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestLead.java b/core/trino-main/src/test/java/io/trino/sql/query/TestLead.java index 01e63fde89ca4..88d79054083b1 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestLead.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestLead.java @@ -15,7 +15,7 @@ import org.junit.jupiter.api.Test; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThat; public class TestLead { @@ -23,11 +23,12 @@ public class TestLead public void testNullOffset() { try (QueryAssertions assertions = new QueryAssertions()) { - assertThatThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" SELECT lead(v, null) OVER (ORDER BY k) FROM (VALUES (1, 10), (2, 20)) t(k, v) """)) - .hasMessageMatching("Offset must not be null"); + .failure() + .hasMessageMatching("Offset must not be null"); } } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestListagg.java b/core/trino-main/src/test/java/io/trino/sql/query/TestListagg.java index 94cd42a0ba348..3e9f05cfe20fc 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestListagg.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestListagg.java @@ -21,7 +21,6 @@ import static io.trino.spi.StandardErrorCode.EXCEEDED_FUNCTION_MEMORY_LIMIT; import static io.trino.spi.StandardErrorCode.SYNTAX_ERROR; import static io.trino.spi.block.PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -168,51 +167,58 @@ public void testListaggQueryWithNullValuesGrouping() public void testListaggQueryIncorrectSyntax() { // missing WITHIN GROUP (ORDER BY ...) - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(v, ',') " + "FROM (VALUES 'a') t(v)")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:24: mismatched input 'FROM'. Expecting: 'WITHIN'"); // missing WITHIN GROUP (ORDER BY ...) - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(v) " + "FROM (VALUES 'a') t(v)")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:19: mismatched input 'FROM'. Expecting: 'WITHIN'"); // too many arguments - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(v, ',', '...') WITHIN GROUP (ORDER BY v)" + "FROM (VALUES 'a') t(v)")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:22: mismatched input ','. Expecting: ')', 'ON'"); // window frames are not supported - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(v, ',') WITHIN GROUP (ORDER BY v) OVER (PARTITION BY id)" + "FROM (VALUES (1, 'a')) t(id, v)")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:55: mismatched input '('. Expecting: ',', 'EXCEPT', 'FETCH', 'FROM', 'GROUP', 'HAVING', 'INTERSECT', 'LIMIT', 'OFFSET', 'ORDER', 'UNION', 'WHERE', 'WINDOW', "); // invalid argument for ON OVERFLOW clause - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(v, ',' ON OVERFLOW COLLAPSE) WITHIN GROUP (ORDER BY v)" + "FROM (VALUES 'a') t(v)")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:35: mismatched input 'COLLAPSE'. Expecting: 'ERROR', 'TRUNCATE'"); // invalid separator type (integer instead of varchar) - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT LISTAGG(v, 123) WITHIN GROUP (ORDER BY v) " + "FROM (VALUES 'Trino', 'SQL', 'everything') t(v) ")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:19: mismatched input '123'. Expecting: "); // invalid truncation filler type (integer instead of varchar) - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT LISTAGG(v, ',' ON OVERFLOW TRUNCATE 1234567890 WITHOUT COUNT) WITHIN GROUP (ORDER BY v) " + "FROM (VALUES 'Trino', 'SQL', 'everything') t(v) ")) + .failure() .hasErrorCode(SYNTAX_ERROR) .hasMessage("line 1:44: mismatched input '1234567890'. Expecting: 'WITH', 'WITHOUT', "); } @@ -221,31 +227,31 @@ public void testListaggQueryIncorrectSyntax() public void testListaggQueryIncorrectExpression() { // integer values - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(value, ',') WITHIN GROUP (ORDER BY value)" + "FROM (VALUES 1, NULL, 2, 3, 4) t(value)")) - .hasMessage("line 1:8: Expected expression of varchar, but 'value' has integer type"); + .failure().hasMessage("line 1:8: Expected expression of varchar, but 'value' has integer type"); // boolean values - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(value, ',') WITHIN GROUP (ORDER BY value)" + "FROM (VALUES TRUE, NULL, FALSE, FALSE, TRUE) t(value)")) - .hasMessage("line 1:8: Expected expression of varchar, but 'value' has boolean type"); + .failure().hasMessage("line 1:8: Expected expression of varchar, but 'value' has boolean type"); // array values - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(value, ',') WITHIN GROUP (ORDER BY value)" + "FROM (VALUES array['abc', 'def'], array['sql']) t(value)")) - .hasMessage("line 1:8: Expected expression of varchar, but 'value' has array(varchar(3)) type"); + .failure().hasMessage("line 1:8: Expected expression of varchar, but 'value' has array(varchar(3)) type"); } @Test public void testListaaggQueryIncorrectOrderByExpression() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT listagg(label, ',') WITHIN GROUP (ORDER BY rgb) " + "FROM (VALUES ('red', rgb(255, 0, 0)), ('green', rgb(0, 128, 0)), ('blue', rgb(0, 0, 255))) color(label, rgb) ")) - .hasMessage("line 1:8: ORDER BY can only be applied to orderable types (actual: color)"); + .failure().hasMessage("line 1:8: ORDER BY can only be applied to orderable types (actual: color)"); } @Test @@ -319,9 +325,10 @@ public void testListaggQueryWithOrderingAndGrouping() @Test public void testListaggQueryOverflowError() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT LISTAGG(value, ',' ON OVERFLOW ERROR) WITHIN GROUP (ORDER BY value) " + "FROM (VALUES lpad('a', " + DEFAULT_MAX_PAGE_SIZE_IN_BYTES + ", 'a'),'Trino') t(value) ")) + .failure() .hasMessage("Concatenated string has the length in bytes larger than the maximum output length 1048576") .hasErrorCode(EXCEEDED_FUNCTION_MEMORY_LIMIT); } @@ -329,7 +336,7 @@ public void testListaggQueryOverflowError() @Test public void testListaggQueryOverflowErrorGrouping() { - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT id, LISTAGG(value, ',' ON OVERFLOW ERROR) WITHIN GROUP (ORDER BY value) " + "FROM (VALUES " + " (1, lpad('a', " + DEFAULT_MAX_PAGE_SIZE_IN_BYTES + ", 'a'))," + @@ -338,6 +345,7 @@ public void testListaggQueryOverflowErrorGrouping() " ) t(id, value) " + "GROUP BY id " + "ORDER BY id ")) + .failure() .hasCauseMessageContaining("Concatenated string has the length in bytes larger than the maximum output length 1048576") .hasErrorCode(EXCEEDED_FUNCTION_MEMORY_LIMIT); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestOrderedAggregation.java b/core/trino-main/src/test/java/io/trino/sql/query/TestOrderedAggregation.java index 71a0721c709dc..f76431cddad51 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestOrderedAggregation.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestOrderedAggregation.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -104,21 +103,21 @@ public void testAggregationWithOrderBy() "SELECT x, y, array_agg(z ORDER BY z) FROM (VALUES (1, 2, 3), (1, 2, 1), (2, 1, 3), (2, 1, 4)) t(x, y, z) GROUP BY GROUPING SETS ((x), (x, y))")) .matches("VALUES (1, NULL, ARRAY[1, 3]), (2, NULL, ARRAY[3, 4]), (1, 2, ARRAY[1, 3]), (2, 1, ARRAY[3, 4])"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT array_agg(z ORDER BY z) OVER (PARTITION BY x) FROM (VALUES (1, 2, 3), (1, 2, 1), (2, 1, 3), (2, 1, 4)) t(x, y, z) GROUP BY x, z")) - .hasMessageMatching(".* Window function with ORDER BY is not supported"); + .failure().hasMessageMatching(".* Window function with ORDER BY is not supported"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT array_agg(DISTINCT x ORDER BY y) FROM (VALUES (1, 2), (3, 5), (4, 1)) t(x, y)")) - .hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); + .failure().hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT array_agg(DISTINCT x+y ORDER BY y) FROM (VALUES (1, 2), (3, 5), (4, 1)) t(x, y)")) - .hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); + .failure().hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT x, array_agg(DISTINCT y ORDER BY z + y DESC) FROM (VALUES (1, 2, 2), (2, 2, 3), (2, 4, 5), (3, 4, 4), (3, 2, 1), (1, 1, 1)) t(x, y, z) GROUP BY x")) - .hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); + .failure().hasMessageMatching(".* For aggregate function with DISTINCT, ORDER BY expressions must appear in arguments"); assertThat(assertions.query( "SELECT multimap_agg(x, y ORDER BY z) FROM (VALUES (1, 2, 2), (1, 5, 5), (2, 1, 5), (3, 4, 4), (2, 5, 1), (1, 1, 1)) t(x, y, z)")) diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestRecursiveCte.java b/core/trino-main/src/test/java/io/trino/sql/query/TestRecursiveCte.java index 49a0e5047e821..d6f7b1412c321 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestRecursiveCte.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestRecursiveCte.java @@ -22,7 +22,6 @@ import static io.trino.SystemSessionProperties.MAX_RECURSION_DEPTH; import static io.trino.SystemSessionProperties.getMaxRecursionDepth; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -276,13 +275,13 @@ public void testSetOperation() @Test public void testRecursionDepthLimitExceeded() { - assertThatThrownBy(() -> assertions.query("WITH RECURSIVE t(n) AS (" + + assertThat(assertions.query("WITH RECURSIVE t(n) AS (" + " SELECT 1" + " UNION ALL" + " SELECT * FROM t" + " )" + " SELECT * FROM t")) - .hasMessage("Recursion depth limit exceeded (%s). Use 'max_recursion_depth' session property to modify the limit.", getMaxRecursionDepth(assertions.getDefaultSession())); + .failure().hasMessage("Recursion depth limit exceeded (%s). Use 'max_recursion_depth' session property to modify the limit.", getMaxRecursionDepth(assertions.getDefaultSession())); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestRowFilter.java b/core/trino-main/src/test/java/io/trino/sql/query/TestRowFilter.java index e57e63afc8053..d7cf17bbb9884 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestRowFilter.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestRowFilter.java @@ -50,9 +50,7 @@ import static io.trino.spi.StandardErrorCode.PERMISSION_DENIED; import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -334,7 +332,8 @@ public void testRecursion() .expression("orderkey IN (SELECT orderkey FROM orders)") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(INVALID_ROW_FILTER) .hasMessageMatching(".*\\QRow filter for 'local.tiny.orders' is recursive\\E.*"); @@ -348,7 +347,8 @@ public void testRecursion() .schema("tiny") .expression("orderkey IN (SELECT local.tiny.orderkey FROM orders)") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(INVALID_ROW_FILTER) .hasMessageMatching(".*\\QRow filter for 'local.tiny.orders' is recursive\\E.*"); @@ -372,7 +372,8 @@ public void testRecursion() .expression("orderkey IN (SELECT orderkey FROM orders)") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(INVALID_ROW_FILTER) .hasMessageMatching(".*\\QRow filter for 'local.tiny.orders' is recursive\\E.*"); } @@ -389,8 +390,9 @@ public void testLimitedScope() .schema("tiny") .expression("orderkey = 1") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT min(name) FROM customer WHERE customer.custkey = orders.custkey) FROM orders")) + .failure() .hasErrorCode(COLUMN_NOT_FOUND) .hasMessage("line 1:31: Invalid row filter for 'local.tiny.customer': Column 'orderkey' cannot be resolved"); } @@ -427,7 +429,8 @@ public void testInvalidFilter() .expression("$$$") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(INVALID_ROW_FILTER) .hasMessage("line 1:22: Invalid row filter for 'local.tiny.orders': mismatched input '$'. Expecting: "); @@ -442,7 +445,8 @@ public void testInvalidFilter() .expression("unknown_column") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(COLUMN_NOT_FOUND) .hasMessage("line 1:22: Invalid row filter for 'local.tiny.orders': Column 'unknown_column' cannot be resolved"); @@ -457,7 +461,8 @@ public void testInvalidFilter() .expression("1") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(TYPE_MISMATCH) .hasMessage("line 1:22: Expected row filter for 'local.tiny.orders' to be of type BOOLEAN, but was integer"); @@ -472,7 +477,8 @@ public void testInvalidFilter() .expression("count(*) > 0") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(EXPRESSION_NOT_SCALAR) .hasMessage("line 1:10: Row filter for 'local.tiny.orders' cannot contain aggregations, window functions or grouping operations: [count(*)]"); @@ -487,7 +493,8 @@ public void testInvalidFilter() .expression("row_number() OVER () > 0") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(EXPRESSION_NOT_SCALAR) .hasMessage("line 1:22: Row filter for 'local.tiny.orders' cannot contain aggregations, window functions or grouping operations: [row_number() OVER ()]"); @@ -502,7 +509,8 @@ public void testInvalidFilter() .expression("grouping(orderkey) = 0") .build()); - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM orders")) + assertThat(assertions.query("SELECT count(*) FROM orders")) + .failure() .hasErrorCode(EXPRESSION_NOT_SCALAR) .hasMessage("line 1:20: Row filter for 'local.tiny.orders' cannot contain aggregations, window functions or grouping operations: [GROUPING (orderkey)]"); } @@ -574,26 +582,30 @@ public void testMergeDelete() ViewExpression.builder().expression("nationkey < 10").build()); // Within allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2) t(x) ON nationkey = x WHEN MATCHED THEN DELETE""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); // Outside allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3,4,5) t(x) ON regionkey = x WHEN MATCHED THEN DELETE""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,11) t(x) ON nationkey = x WHEN MATCHED THEN DELETE""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 11,12,13,14,15) t(x) ON nationkey = x WHEN MATCHED THEN DELETE""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); } @@ -611,38 +623,47 @@ public void testUpdate() ViewExpression.builder().expression("nationkey < 10").build()); // Within allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey < 3")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey < 3")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey IN (1, 2, 3)")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey IN (1, 2, 3)")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); // Outside allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey IN (1, 11)")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey IN (1, 11)")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey = 11")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET regionkey = regionkey * 2 WHERE nationkey = 11")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); // Within allowed row filter, but updated rows are outside the row filter - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET nationkey = 10 WHERE nationkey < 3")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET nationkey = 10 WHERE nationkey < 3")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET nationkey = null WHERE nationkey < 3")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET nationkey = null WHERE nationkey < 3")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); // Outside allowed row filter, and updated rows are outside the row filter - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET nationkey = 10 WHERE nationkey = 10")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET nationkey = 10 WHERE nationkey = 10")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation SET nationkey = null WHERE nationkey = null ")) + assertThat(assertions.query("UPDATE mock.tiny.nation SET nationkey = null WHERE nationkey = null ")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Updating a table with a row filter is not supported"); } @@ -660,55 +681,64 @@ public void testMergeUpdate() ViewExpression.builder().expression("nationkey < 10").build()); // Within allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 5) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 2""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); // Outside allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3,4,5,6) t(x) ON regionkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 2""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1, 11) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 2""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 11) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET regionkey = regionkey * 2""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); // Within allowed row filter, but updated rows are outside the row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET nationkey = 10""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 1,2,3) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET nationkey = NULL""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); // Outside allowed row filter, but updated rows are outside the row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 10) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET nationkey = 13""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 10) t(x) ON nationkey = x WHEN MATCHED THEN UPDATE SET nationkey = NULL""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 10) t(x) ON nationkey IS NULL WHEN MATCHED THEN UPDATE SET nationkey = 13""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); } @@ -732,18 +762,22 @@ public void testInsert() .matches("SELECT BIGINT '1'"); // Outside allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 0, 'No comment')")) + assertThat(assertions.query("INSERT INTO mock.tiny.nation VALUES (26, 'POLAND', 0, 'No comment')")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation VALUES " + assertThat(assertions.query("INSERT INTO mock.tiny.nation VALUES " + "(26, 'POLAND', 0, 'No comment')," + "(27, 'HOLLAND', 0, 'A comment')")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation(nationkey) VALUES (null)")) + assertThat(assertions.query("INSERT INTO mock.tiny.nation(nationkey) VALUES (null)")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation(regionkey) VALUES (0)")) + assertThat(assertions.query("INSERT INTO mock.tiny.nation(regionkey) VALUES (0)")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); } @@ -761,32 +795,37 @@ public void testMergeInsert() ViewExpression.builder().expression("nationkey > 100").build()); // Within allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 42) t(dummy) ON false WHEN NOT MATCHED THEN INSERT VALUES (101, 'POLAND', 0, 'No comment')""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); // Outside allowed row filter - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 42) t(dummy) ON false WHEN NOT MATCHED THEN INSERT VALUES (26, 'POLAND', 0, 'No comment')""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES (26, 'POLAND', 0, 'No comment'), (27, 'HOLLAND', 0, 'A comment')) t(a,b,c,d) ON nationkey = a WHEN NOT MATCHED THEN INSERT VALUES (a,b,c,d)""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 42) t(dummy) ON false WHEN NOT MATCHED THEN INSERT (nationkey) VALUES (NULL)""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); - assertTrinoExceptionThrownBy(() -> assertions.query(""" + assertThat(assertions.query(""" MERGE INTO mock.tiny.nation USING (VALUES 42) t(dummy) ON false WHEN NOT MATCHED THEN INSERT (nationkey) VALUES (0)""")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessage("line 1:1: Cannot merge into a table with row filters"); } @@ -804,14 +843,16 @@ public void testRowFilterWithHiddenColumns() .assertThat() .skippingTypesCheck() .matches("VALUES (BIGINT '0', 'ALGERIA', BIGINT '0', ' haggle. carefully final deposits detect slyly agai')"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column VALUES (101, 'POLAND', 0, 'No comment')")) + assertThat(assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column VALUES (101, 'POLAND', 0, 'No comment')")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column VALUES (0, 'POLAND', 0, 'No comment')") .assertThat() .skippingTypesCheck() .matches("VALUES BIGINT '1'"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'POLAND'")) + assertThat(assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'POLAND'")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessageContaining("Updating a table with a row filter is not supported"); assertions.query("DELETE FROM mock.tiny.nation_with_hidden_column WHERE regionkey < 5") @@ -838,10 +879,12 @@ public void testRowFilterOnHiddenColumn() .skippingTypesCheck() .matches("VALUES BIGINT '25'"); // TODO https://github.com/trinodb/trino/issues/10006 - support insert into a table with row filter that is using hidden columns - assertThatThrownBy(() -> assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column VALUES (101, 'POLAND', 0, 'No comment')")) + assertThat(assertions.query("INSERT INTO mock.tiny.nation_with_hidden_column VALUES (101, 'POLAND', 0, 'No comment')")) // TODO this should be TrinoException (assertTrinoExceptionThrownBy) + .nonTrinoExceptionFailure() .hasStackTraceContaining("ArrayIndexOutOfBoundsException: Index 4 out of bounds for length 4"); - assertTrinoExceptionThrownBy(() -> assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'POLAND'")) + assertThat(assertions.query("UPDATE mock.tiny.nation_with_hidden_column SET name = 'POLAND'")) + .failure() .hasErrorCode(NOT_SUPPORTED) .hasMessageContaining("Updating a table with a row filter is not supported"); assertions.query("DELETE FROM mock.tiny.nation_with_hidden_column WHERE regionkey < 5") @@ -865,11 +908,13 @@ public void testRowFilterOnOptionalColumn() .skippingTypesCheck() .matches("VALUES BIGINT '1'"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mockmissingcolumns.tiny.nation_with_optional_column(nationkey, name, regionkey, comment, optional) VALUES (0, 'POLAND', 0, 'No comment', 'so')")) + assertThat(assertions.query("INSERT INTO mockmissingcolumns.tiny.nation_with_optional_column(nationkey, name, regionkey, comment, optional) VALUES (0, 'POLAND', 0, 'No comment', 'so')")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); - assertTrinoExceptionThrownBy(() -> assertions.query("INSERT INTO mockmissingcolumns.tiny.nation_with_optional_column(nationkey, name, regionkey, comment, optional) VALUES (0, 'POLAND', 0, 'No comment', null)")) + assertThat(assertions.query("INSERT INTO mockmissingcolumns.tiny.nation_with_optional_column(nationkey, name, regionkey, comment, optional) VALUES (0, 'POLAND', 0, 'No comment', null)")) + .failure() .hasErrorCode(PERMISSION_DENIED) .hasMessage("Access Denied: Cannot insert row that does not match a row filter"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatching.java b/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatching.java index 43dc05412632b..99688ae09ba57 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatching.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatching.java @@ -20,7 +20,6 @@ import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -818,12 +817,12 @@ public void testAfterMatchSkip() " (6, 2, 80, 'C') "); // Exception: trying to resume matching from the first row of the match. If uncaught, it would cause an infinite loop. - assertThatThrownBy(() -> assertions.query(format(query, "AFTER MATCH SKIP TO A"))) - .hasMessage("AFTER MATCH SKIP failed: cannot skip to first row of match"); + assertThat(assertions.query(format(query, "AFTER MATCH SKIP TO A"))) + .failure().hasMessage("AFTER MATCH SKIP failed: cannot skip to first row of match"); // Exception: trying to skip to label which was not matched - assertThatThrownBy(() -> assertions.query(format(query, "AFTER MATCH SKIP TO D"))) - .hasMessage("AFTER MATCH SKIP failed: pattern variable is not present in match"); + assertThat(assertions.query(format(query, "AFTER MATCH SKIP TO D"))) + .failure().hasMessage("AFTER MATCH SKIP failed: pattern variable is not present in match"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatchingInWindow.java b/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatchingInWindow.java index deecd41790d2c..52c3d46161385 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatchingInWindow.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestRowPatternMatchingInWindow.java @@ -20,7 +20,6 @@ import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -660,12 +659,12 @@ public void testAfterMatchSkipToLabel() " (6, null, null) "); // Exception: trying to resume matching from the first row of the match. If uncaught, it would cause an infinite loop. - assertThatThrownBy(() -> assertions.query(format(query, "AFTER MATCH SKIP TO A"))) - .hasMessage("AFTER MATCH SKIP failed: cannot skip to first row of match"); + assertThat(assertions.query(format(query, "AFTER MATCH SKIP TO A"))) + .failure().hasMessage("AFTER MATCH SKIP failed: cannot skip to first row of match"); // Exception: trying to skip to label which was not matched - assertThatThrownBy(() -> assertions.query(format(query, "AFTER MATCH SKIP TO D"))) - .hasMessage("AFTER MATCH SKIP failed: pattern variable is not present in match"); + assertThat(assertions.query(format(query, "AFTER MATCH SKIP TO D"))) + .failure().hasMessage("AFTER MATCH SKIP failed: pattern variable is not present in match"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestSelectAll.java b/core/trino-main/src/test/java/io/trino/sql/query/TestSelectAll.java index b7574a09caa65..23007e8a8b6c1 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestSelectAll.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestSelectAll.java @@ -20,7 +20,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -120,16 +119,16 @@ public void testSelectAllWithOrderBy() "SELECT t.r.* FROM (VALUES ROW(CAST(ROW(1) AS ROW(f1 integer))), ROW(CAST(ROW(2) AS ROW(f1 integer)))) t(r) ORDER BY f1 DESC")) .ordered() .matches("VALUES 2, 1"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT t.r.* FROM (VALUES ROW(CAST(ROW(1) AS ROW(f1 integer))), ROW(CAST(ROW(2) AS ROW(f2 integer)))) t(r) ORDER BY f1 DESC")) - .hasMessageMatching(".*Column 'f1' cannot be resolved"); + .failure().hasMessageMatching(".*Column 'f1' cannot be resolved"); assertThat(assertions.query( "SELECT t.r.* AS (f1) FROM (VALUES ROW(CAST(ROW(1) AS ROW(f1 integer))), ROW(CAST(ROW(2) AS ROW(f2 integer)))) t(r) ORDER BY f1 DESC")) .ordered() .matches("VALUES 2, 1"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT t.r.* AS (x) FROM (VALUES ROW(CAST(ROW(1) AS ROW(f1 bigint))), ROW(CAST(ROW(2) AS ROW(f1 bigint)))) t(r) ORDER BY f1 DESC")) - .hasMessageMatching(".*Column 'f1' cannot be resolved"); + .failure().hasMessageMatching(".*Column 'f1' cannot be resolved"); // order by row assertThat(assertions.query( @@ -198,18 +197,18 @@ public void testSelectAllFromOuterScopeTable() "CAST(ROW(ROW(1, 1)) AS row(row(a integer, b integer))), " + "CAST(ROW(ROW(2, 2)) AS row(row(a integer, b integer)))"); // the following query should fail due to multiple rows returned from subquery, but instead fails to decorrelate - assertThatThrownBy(() -> assertions.query("SELECT (SELECT t.* FROM (VALUES 0, 1)) FROM (VALUES 2) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0, 1)) FROM (VALUES 2) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // filter in subquery assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) WHERE true) FROM (VALUES 1) t(a)")).matches("VALUES 1"); assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) WHERE 0 = 0) FROM (VALUES 1) t(a)")).matches("VALUES 1"); - assertThatThrownBy(() -> assertions.query("SELECT (SELECT t.* FROM (VALUES 0) t2(b) WHERE b > 1) FROM (VALUES 1) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); - assertThatThrownBy(() -> assertions.query("SELECT (SELECT t.* FROM (VALUES 0) WHERE false) FROM (VALUES 1) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) t2(b) WHERE b > 1) FROM (VALUES 1) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) WHERE false) FROM (VALUES 1) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // limit in subquery assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) LIMIT 1) FROM (VALUES 1, 2) t(a)")).matches("VALUES 1, 2"); assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) LIMIT 5) FROM (VALUES 1, 2) t(a)")).matches("VALUES 1, 2"); assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) LIMIT 0) FROM (VALUES 1, 2) t(a)")).matches("VALUES CAST(NULL AS INTEGER), CAST(NULL AS INTEGER)"); - assertThatThrownBy(() -> assertions.query("SELECT (SELECT t.* FROM (VALUES 0, 1) LIMIT 1) FROM (VALUES 2, 3) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); - assertThatThrownBy(() -> assertions.query("SELECT (SELECT t.* FROM (SELECT * FROM (VALUES 0, 1) LIMIT 1)) FROM (VALUES 2, 3) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0, 1) LIMIT 1) FROM (VALUES 2, 3) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT (SELECT t.* FROM (SELECT * FROM (VALUES 0, 1) LIMIT 1)) FROM (VALUES 2, 3) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // alias shadowing assertThat(assertions.query("SELECT (SELECT t.* FROM (VALUES 0) t) FROM (VALUES 1) t(a)")).matches("VALUES 0"); assertThat(assertions.query("SELECT(SELECT(SELECT t.* FROM (VALUES 0)) FROM (VALUES 1) t(a)) FROM (VALUES 2) t(a)")).matches("VALUES 1"); @@ -219,31 +218,31 @@ public void testSelectAllFromOuterScopeTable() assertThat(assertions.query("SELECT EXISTS(SELECT t.* FROM (VALUES 1) t2(b) WHERE t2.b > t.a) FROM (VALUES 0, 2) t(a)")).matches("VALUES true, false"); // IN subquery - assertThatThrownBy(() -> assertions.query("SELECT 1 IN (SELECT t.*) FROM (VALUES 1, 2) t(a)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT 1 IN (SELECT t.*) FROM (VALUES 1, 2) t(a)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // lateral relation assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.*)")).matches("VALUES (0, 0), (1, 1)"); assertThat(assertions.query("SELECT t.a, t2.d FROM (VALUES (0, 1), (2, 3)) t(a, b), LATERAL (SELECT t.*) t2(c, d)")).matches("VALUES (0, 1), (2, 3)"); // limit in lateral relation assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* LIMIT 5)")).matches("VALUES (0, 0), (1, 1)"); - assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* LIMIT 0)")).matches(result -> result.getMaterializedRows().isEmpty()); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* LIMIT 0)")).result().isEmpty(); // filter in lateral relation assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE true)")).matches("VALUES (0, 0), (1, 1)"); assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE 0 = 0)")).matches("VALUES (0, 0), (1, 1)"); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE false)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE t.a = 0)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE false)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* WHERE t.a = 0)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // FROM in lateral relation assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (VALUES 1))")).matches("VALUES (0, 0), (1, 1)"); assertThat(assertions.query("SELECT t.* FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.*) t")).matches("VALUES (0, 0), (1, 1)"); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (VALUES 1, 2))")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (VALUES 1, 2) LIMIT 1)")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (SELECT * FROM (VALUES 1, 2) LIMIT 1))")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (VALUES 1, 2))")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (VALUES 1, 2) LIMIT 1)")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t.* FROM (SELECT * FROM (VALUES 1, 2) LIMIT 1))")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); // reference to further outer scope relation assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t2.* from (VALUES 3, 4) t2(b), LATERAL (SELECT t.*))")).matches("VALUES (0, 3), (1, 3), (0, 4), (1, 4)"); assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t2.* from (VALUES 2), LATERAL (SELECT t.*) t2(b))")).matches("VALUES (0, 0), (1, 1)"); assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT t2.b from (VALUES 2), LATERAL (SELECT t.*) t2(b))")).matches("VALUES (0, 0), (1, 1)"); - assertThatThrownBy(() -> assertions.query("SELECT * FROM (VALUES 0) t(a), LATERAL (SELECT t2.* from (VALUES 1, 2), LATERAL (SELECT t.*) t2(b))")).hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); + assertThat(assertions.query("SELECT * FROM (VALUES 0) t(a), LATERAL (SELECT t2.* from (VALUES 1, 2), LATERAL (SELECT t.*) t2(b))")).failure().hasMessageMatching(UNSUPPORTED_DECORRELATION_MESSAGE); assertThat(assertions.query("SELECT * FROM (VALUES 0, 1) t(a), LATERAL (SELECT * from (VALUES 2), LATERAL (SELECT t.*))")).matches("VALUES (0, 2, 0), (1, 2, 1)"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestShowQueries.java b/core/trino-main/src/test/java/io/trino/sql/query/TestShowQueries.java index f1df341868e87..65231baf7238f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestShowQueries.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestShowQueries.java @@ -28,7 +28,6 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -76,10 +75,10 @@ public void teardown() @Test public void testShowCatalogsLikeWithEscape() { - assertThatThrownBy(() -> assertions.query("SHOW CATALOGS LIKE 't$_%' ESCAPE ''")) - .hasMessage("Escape string must be a single character"); - assertThatThrownBy(() -> assertions.query("SHOW CATALOGS LIKE 't$_%' ESCAPE '$$'")) - .hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW CATALOGS LIKE 't$_%' ESCAPE ''")) + .failure().hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW CATALOGS LIKE 't$_%' ESCAPE '$$'")) + .failure().hasMessage("Escape string must be a single character"); assertThat(assertions.query("SHOW CATALOGS LIKE '%$_%' ESCAPE '$'")).matches("VALUES('testing_catalog')"); assertThat(assertions.query("SHOW CATALOGS LIKE '$_%' ESCAPE '$'")).matches("SELECT 'testing_catalog' WHERE FALSE"); } @@ -119,10 +118,10 @@ public void testShowSessionLike() @Test public void testShowSessionLikeWithEscape() { - assertThatThrownBy(() -> assertions.query("SHOW SESSION LIKE 't$_%' ESCAPE ''")) - .hasMessage("Escape string must be a single character"); - assertThatThrownBy(() -> assertions.query("SHOW SESSION LIKE 't$_%' ESCAPE '$$'")) - .hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW SESSION LIKE 't$_%' ESCAPE ''")) + .failure().hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW SESSION LIKE 't$_%' ESCAPE '$$'")) + .failure().hasMessage("Escape string must be a single character"); assertThat(assertions.query( "SHOW SESSION LIKE '%page$_row$_c%' ESCAPE '$'")) .skippingTypesCheck() @@ -176,10 +175,10 @@ public void testShowColumnsLike() @Test public void testShowColumnsWithLikeWithEscape() { - assertThatThrownBy(() -> assertions.query("SHOW COLUMNS FROM system.runtime.nodes LIKE 't$_%' ESCAPE ''")) - .hasMessage("Escape string must be a single character"); - assertThatThrownBy(() -> assertions.query("SHOW COLUMNS FROM system.runtime.nodes LIKE 't$_%' ESCAPE '$$'")) - .hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW COLUMNS FROM system.runtime.nodes LIKE 't$_%' ESCAPE ''")) + .failure().hasMessage("Escape string must be a single character"); + assertThat(assertions.query("SHOW COLUMNS FROM system.runtime.nodes LIKE 't$_%' ESCAPE '$$'")) + .failure().hasMessage("Escape string must be a single character"); assertThat(assertions.query("SHOW COLUMNS FROM mock.mockSchema.mockTable LIKE 'cola$_' ESCAPE '$'")) .matches("VALUES (VARCHAR 'cola_', VARCHAR 'bigint' , VARCHAR '', VARCHAR '')"); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestSubqueries.java b/core/trino-main/src/test/java/io/trino/sql/query/TestSubqueries.java index 761f31da754de..a8d1bb816da7f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestSubqueries.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestSubqueries.java @@ -53,7 +53,6 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -134,9 +133,9 @@ public void testSubqueriesWithGroupByAndCoercions() "SELECT (SELECT count(*) FROM (VALUES 1, 2, 2) t(a) WHERE t.a=t2.b GROUP BY t.a LIMIT 1) FROM (VALUES 1.0) t2(b)")) .matches("VALUES BIGINT '1'"); // non-injective coercion bigint -> double - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT EXISTS(SELECT 1 FROM (VALUES (BIGINT '1', null)) t(a, b) WHERE t.a=t2.b GROUP BY t.b) FROM (VALUES 1e0, 2e0) t2(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -171,9 +170,9 @@ public void testCorrelatedSubqueriesWithLimitOne() .matches("VALUES 1, 5"); // non-injective coercion bigint -> double - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES BIGINT '1', BIGINT '2') t(a) WHERE t.a = t2.b LIMIT 1) FROM (VALUES 1e0, 2e0) t2(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -191,9 +190,9 @@ public void testCorrelatedSubqueriesWithLimitGreaterThanOne() .matches("VALUES 1, 5"); // non-injective coercion bigint -> double - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES BIGINT '1', BIGINT '2', BIGINT '3') t(a) WHERE t.a = t2.b LIMIT 2) FROM (VALUES 1e0, 2e0) t2(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -246,9 +245,9 @@ public void testCorrelatedSubqueriesWithTopN() values("a")))))))); // non-injective coercion bigint -> double - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES BIGINT '1', BIGINT '2') t(a) WHERE t.a = t2.b ORDER BY a LIMIT 1) FROM (VALUES 1e0, 2e0) t2(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -263,28 +262,28 @@ public void testCorrelatedSubqueriesWithLimit() assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES 1, 2, 3) t(a) WHERE t.a = t2.b LIMIT 2) FROM (VALUES 1) t2(b)")) .matches("VALUES 1"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES 1, 1, 2, 3) t(a) WHERE t.a = t2.b LIMIT 2) FROM (VALUES 1) t2(b)")) - .hasMessageMatching("Scalar sub-query has returned multiple rows"); + .failure().hasMessageMatching("Scalar sub-query has returned multiple rows"); // Limit(1) and non-constant output symbol of the subquery - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT count(*) FROM (VALUES (1, 0), (1, 1)) t(a, b) WHERE a = c GROUP BY b LIMIT 1) FROM (VALUES (1)) t2(c)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); // Limit(1) and non-constant output symbol of the subquery - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT a + b FROM (VALUES (1, 1), (1, 1)) t(a, b) WHERE a = c LIMIT 1) FROM (VALUES (1)) t2(c)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); // Limit and correlated non-equality predicate in the subquery - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.b FROM (VALUES (1, 2), (1, 3)) t(a, b) WHERE t.a = t2.a AND t.b > t2.b LIMIT 1) FROM (VALUES (1, 2)) t2(a, b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES (1, 2), (1, 3)) t(a, b) WHERE t.a = t2.a AND t2.b > 1 LIMIT 1) FROM (VALUES (1, 2)) t2(a, b)")) .matches("VALUES 1"); // TopN and correlated non-equality predicate in the subquery - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.b FROM (VALUES (1, 2), (1, 3)) t(a, b) WHERE t.a = t2.a AND t.b > t2.b ORDER BY t.b LIMIT 1) FROM (VALUES (1, 2)) t2(a, b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); assertThat(assertions.query( "SELECT (SELECT t.b FROM (VALUES (1, 2), (1, 3)) t(a, b) WHERE t.a = t2.a AND t2.b > 1 ORDER BY t.b LIMIT 1) FROM (VALUES (1, 2)) t2(a, b)")) .matches("VALUES 2"); @@ -327,9 +326,9 @@ public void testCorrelatedSubqueriesWithLimit() anyTree( values("u_x", "u_cid"))))))))); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT t.a FROM (VALUES 1, 2, 3) t(a) WHERE t.a = t2.b ORDER BY a FETCH FIRST ROW WITH TIES) FROM (VALUES 1) t2(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); assertThat(assertions.query( "SELECT * " + "FROM (VALUES 1, 2, 3, null) outer_relation(id) " + @@ -400,14 +399,14 @@ public void testCorrelatedSubqueriesWithLimit() "ON TRUE")) .matches("VALUES (1, null), (2, null), (3, 'a'), (3, 'b'), (null, null)"); // TopN with ordering not decorrelating - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * " + "FROM (VALUES 1, 2, 3, null) outer_relation(id) " + "LEFT JOIN LATERAL " + "(SELECT value FROM (VALUES 'c', 'a', 'b') inner_relation(value) " + " WHERE outer_relation.id = 3 ORDER BY outer_relation.id LIMIT 2) " + "ON TRUE")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); // TopN with ordering only by constants assertThat(assertions.query( "SELECT * " + @@ -458,9 +457,9 @@ public void testNestedUncorrelatedSubqueryInCorrelatedSubquery() public void testCorrelatedSubqueriesWithGroupBy() { // t.a is not a "constant" column, group by does not guarantee single row per correlated subquery - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT count(*) FROM (VALUES 1, 2, 3, null) t(a) WHERE t.a 1) FROM (VALUES 1, 2) t2(b)")) .matches("VALUES null, BIGINT '2'"); @@ -518,9 +517,9 @@ public void testCorrelatedSubqueriesWithGroupBy() values("t_a", "t_b")))))))))); // t.b is not a "constant" column, cannot be pushed above aggregation - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT EXISTS(SELECT 1 FROM (VALUES (1, 1), (1, 1), (null, null), (3, 3)) t(a, b) WHERE t.a+t.b 1)")) .matches("VALUES (1, BIGINT '2')"); // correlated subqueries with grouping sets are not supported - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, 2) t2(b), LATERAL (SELECT t.a, t.b, count(*) FROM (VALUES (1, 1), (1, 2), (2, 2), (3, 3)) t(a, b) WHERE t.a=t2.b GROUP BY GROUPING SETS ((t.a, t.b), (t.a)))")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -630,9 +629,9 @@ public void testUncorrelatedSubquery() assertThat(assertions.query("SELECT * FROM (VALUES 1, null) t(a) FULL JOIN LATERAL (SELECT * FROM (VALUES 2, null)) t2(b) ON TRUE")).matches("VALUES (1, 2), (1, null), (null, 2), (null, null)"); assertions.assertQueryReturnsEmptyResult("SELECT * FROM (SELECT 1 WHERE 0 = 1) t(a) FULL JOIN LATERAL (SELECT * FROM (VALUES 2, null)) t2(b) ON TRUE"); assertThat(assertions.query("SELECT * FROM (VALUES 1, null) t(a) FULL JOIN LATERAL (SELECT 1 WHERE 0 = 1) t2(b) ON TRUE")).matches("VALUES (1, CAST(null AS INTEGER)), (null, null)"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES 1, null) t(a) FULL JOIN LATERAL (SELECT * FROM (VALUES 2, null)) t2(b) ON a < b")) - .hasMessageMatching(".* FULL JOIN involving LATERAL relation is only supported with condition ON TRUE"); + .failure().hasMessageMatching(".* FULL JOIN involving LATERAL relation is only supported with condition ON TRUE"); } @Test @@ -715,9 +714,9 @@ public void testCorrelatedSubqueryWithoutFilter() assertThat(assertions.query( "SELECT (SELECT a + b FROM (VALUES 1) inner_relation(a)) FROM (VALUES 2) outer_relation(b)")) .matches("VALUES 3"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT rank() OVER(partition by b) FROM (VALUES 1) inner_relation(a)) FROM (VALUES 2) outer_relation(b)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); } @Test @@ -811,15 +810,16 @@ public void testCorrelatedInnerUnnestWithGlobalAggregation() .matches("VALUES ROW(ROW(ARRAY[1, 2, 3], CAST(ARRAY[1, 2, 3] AS array(bigint)))), ROW(ROW(ARRAY[4], ARRAY[1])), ROW(ROW(ARRAY[5, 6], ARRAY[1, 2]))"); // correlated grouping key - illegal by AggregationAnalyzer - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT max(count) FROM (SELECT count(v) AS count FROM UNNEST(id, val) u(i, v) GROUP BY id)) " + "FROM (VALUES (ARRAY['a', 'a', 'b'], ARRAY[1, 2, 3])) t(id, val)")) - .hasMessageMatching("Grouping field .* should originate from .*"); + // TODO this should be TrinoException + .nonTrinoExceptionFailure().hasMessageMatching("Grouping field .* should originate from .*"); // aggregation with filter: all aggregations have filter, so filter is pushed down and it is not supported by the correlated unnest rewrite - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT array_agg(x) FILTER (WHERE x < 3) FROM UNNEST(a) u(x)) FROM (VALUES ARRAY[1, 2, 3]) t(a)")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); // aggregation with filter: no filter pushdown assertThat(assertions.query( @@ -974,22 +974,23 @@ public void testCorrelatedLeftUnnestWithGlobalAggregation() .matches("VALUES ROW(ROW(ARRAY[1, 2, 3], CAST(ARRAY[1, 2, 3] AS array(bigint)))), ROW(ROW(ARRAY[4], ARRAY[1])), ROW(ROW(ARRAY[5, 6], ARRAY[1, 2]))"); // correlated grouping key - illegal by AggregationAnalyzer - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT b FROM " + "(VALUES (ARRAY['a', 'a', 'b'], ARRAY[1, 2, 3])) t(id, val) " + "LEFT JOIN " + "LATERAL (SELECT max(count) FROM (SELECT count(v) AS count FROM (SELECT i, v FROM (VALUES 1) LEFT JOIN UNNEST(id, val) u(i, v) ON TRUE) GROUP BY id)) t2(b) " + "ON TRUE")) - .hasMessageMatching("Grouping field .* should originate from .*"); + // TODO this should be TrinoException + .nonTrinoExceptionFailure().hasMessageMatching("Grouping field .* should originate from .*"); // aggregation with filter: all aggregations have filter, so filter is pushed down and it is not supported by the correlated unnest rewrite - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT b FROM " + "(VALUES ARRAY[1, 2, 3]) t(a) " + "LEFT JOIN " + "LATERAL (SELECT array_agg(x) FILTER (WHERE x < 3) FROM (SELECT x FROM (VALUES 1) LEFT JOIN UNNEST(a) u(x) ON TRUE)) t2(b) " + "ON TRUE")) - .hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); + .failure().hasMessageMatching(UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); // aggregation with filter: no filter pushdown assertThat(assertions.query( @@ -1075,10 +1076,10 @@ public void testCorrelatedUnnestInScalarSubquery() "FROM (VALUES ARRAY[1], ARRAY[2]) t(a)")) .matches("VALUES 1, 2"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT (SELECT * FROM UNNEST(a) u(x)) " + "FROM (VALUES ARRAY[1, 2, 3]) t(a)")) - .hasMessage("Scalar sub-query has returned multiple rows"); + .failure().hasMessage("Scalar sub-query has returned multiple rows"); // limit in subquery assertThat(assertions.query( diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestTDigestFunctions.java b/core/trino-main/src/test/java/io/trino/sql/query/TestTDigestFunctions.java index c8d4a272d01e7..a2853b5b8c8c8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestTDigestFunctions.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestTDigestFunctions.java @@ -25,7 +25,6 @@ import static java.lang.Math.round; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -78,10 +77,10 @@ public void testValuesAtQuantiles() "FROM (VALUES 0.1e0, 0.1e0, 0.1e0, 0.1e0, 10e0) T(d)")) .matches("VALUES ARRAY[0.1e0, 0.1e0, 10.0e0]"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT values_at_quantiles(tdigest_agg(d), ARRAY[1e0, 0e0]) " + "FROM (VALUES 0.1e0) T(d)")) - .hasMessage("percentiles must be sorted in increasing order"); + .failure().hasMessage("percentiles must be sorted in increasing order"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestTrim.java b/core/trino-main/src/test/java/io/trino/sql/query/TestTrim.java index 003b46efc2d7c..9b8d004fddb52 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestTrim.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestTrim.java @@ -22,7 +22,6 @@ import org.junit.jupiter.api.parallel.Execution; import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -329,7 +328,8 @@ private void assertFunction(@Language("SQL") String actual, @Language("SQL") Str private void assertInvalidFunction(@Language("SQL") String actual, String message) { - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT " + actual)) + assertThat(assertions.query("SELECT " + actual)) + .failure() .hasMessage(message) .hasErrorCode(INVALID_FUNCTION_ARGUMENT); } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestUnnest.java b/core/trino-main/src/test/java/io/trino/sql/query/TestUnnest.java index 44c785f143606..78ef65ed36cb4 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestUnnest.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestUnnest.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -66,11 +65,11 @@ public void testUnnestPreserveColumnName() "SELECT x FROM UNNEST(CAST(ARRAY[ROW(1, 'a'), ROW(2, 'b')] as ARRAY(ROW(x int, y varchar))))")) .matches("VALUES (1), (2)"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT x FROM" + "(VALUES (3)) AS t(x)" + "CROSS JOIN UNNEST(CAST(ARRAY[ROW(1, 'a'), ROW(2, 'b')] as ARRAY(ROW(x int, y varchar))))")) - .hasMessageMatching(".*Column 'x' is ambiguous.*"); + .failure().hasMessageMatching(".*Column 'x' is ambiguous.*"); assertThat(assertions.query( "SELECT t.x FROM" + @@ -88,12 +87,12 @@ public void testUnnestPreserveColumnName() @Test public void testUnnestMultiExpr() { - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT x " + "FROM UNNEST(" + " CAST(ARRAY[ROW(1, 'a'), ROW(2, 'b')] as ARRAY(ROW(x int, y varchar)))," + " CAST(ARRAY[ROW(1, 'a'), ROW(2, 'b')] as ARRAY(ROW(x int, y varchar))))")) - .hasMessageMatching(".*Column 'x' is ambiguous.*"); + .failure().hasMessageMatching(".*Column 'x' is ambiguous.*"); assertThat(assertions.query( "SELECT t3 " + @@ -125,9 +124,9 @@ public void testLeftJoinUnnest() assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[]) a(x) LEFT OUTER JOIN UNNEST(x) WITH ORDINALITY ON true")) .matches("VALUES (ARRAY[], null, CAST(NULL AS bigint))"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[1, null]) a(x) LEFT OUTER JOIN UNNEST(x) b(y) ON b.y = 1")) - .hasMessageMatching("line .*: LEFT JOIN involving UNNEST is only supported with condition ON TRUE"); + .failure().hasMessageMatching("line .*: LEFT JOIN involving UNNEST is only supported with condition ON TRUE"); assertThat(assertions.query( "SELECT * FROM (VALUES 'a', 'b') LEFT JOIN UNNEST(ARRAY[]) ON TRUE")) .matches("VALUES ('a', null), ('b', null)"); @@ -156,9 +155,9 @@ public void testRightJoinUnnest() assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[1, null]) a(x) RIGHT OUTER JOIN UNNEST(ARRAY[2, null]) WITH ORDINALITY ON true")) .matches("VALUES (ARRAY[1, null], 2, BIGINT '1'), (ARRAY[1, null], null, BIGINT '2')"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[1, null]) a(x) RIGHT OUTER JOIN UNNEST(ARRAY[2, null]) b(y) ON b.y = 1")) - .hasMessageMatching("line .*: RIGHT JOIN involving UNNEST is only supported with condition ON TRUE"); + .failure().hasMessageMatching("line .*: RIGHT JOIN involving UNNEST is only supported with condition ON TRUE"); } @Test @@ -177,9 +176,9 @@ public void testFullJoinUnnest() assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[]) a(x) FULL OUTER JOIN UNNEST(ARRAY[2, null]) WITH ORDINALITY ON true")) .matches("VALUES (ARRAY[], 2, BIGINT '1'), (ARRAY[], null, BIGINT '2')"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[1, null]) a(x) FULL OUTER JOIN UNNEST(ARRAY[2, null]) b(y) ON b.y = 1")) - .hasMessageMatching("line .*: FULL JOIN involving UNNEST is only supported with condition ON TRUE"); + .failure().hasMessageMatching("line .*: FULL JOIN involving UNNEST is only supported with condition ON TRUE"); } @Test @@ -195,9 +194,9 @@ public void testInnerJoinUnnest() "SELECT * FROM (VALUES ARRAY[]) a(x) INNER JOIN UNNEST(x) ON true"); assertions.assertQueryReturnsEmptyResult( "SELECT * FROM (VALUES ARRAY[]) a(x) INNER JOIN UNNEST(x) WITH ORDINALITY ON true"); - assertThatThrownBy(() -> assertions.query( + assertThat(assertions.query( "SELECT * FROM (VALUES ARRAY[1, null]) a(x) INNER JOIN UNNEST(x) b(y) ON b.y = 1")) - .hasMessageMatching("line .*: INNER JOIN involving UNNEST is only supported with condition ON TRUE"); + .failure().hasMessageMatching("line .*: INNER JOIN involving UNNEST is only supported with condition ON TRUE"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestValues.java b/core/trino-main/src/test/java/io/trino/sql/query/TestValues.java index 27702f59a6b9f..7f59d2d809b0a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestValues.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestValues.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static io.trino.spi.StandardErrorCode.DIVISION_BY_ZERO; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -108,13 +107,13 @@ public void testNulls() @Test public void testFailingExpression() { - assertTrinoExceptionThrownBy(() -> assertions.query("VALUES 0 / 0")) - .hasErrorCode(DIVISION_BY_ZERO); + assertThat(assertions.query("VALUES 0 / 0")) + .failure().hasErrorCode(DIVISION_BY_ZERO); - assertTrinoExceptionThrownBy(() -> assertions.query("VALUES CASE 1 WHEN 0 THEN true WHEN 0 / 0 THEN false END")) - .hasErrorCode(DIVISION_BY_ZERO); + assertThat(assertions.query("VALUES CASE 1 WHEN 0 THEN true WHEN 0 / 0 THEN false END")) + .failure().hasErrorCode(DIVISION_BY_ZERO); - assertTrinoExceptionThrownBy(() -> assertions.query("VALUES IF(0 / 0 > 0, true, false)")) - .hasErrorCode(DIVISION_BY_ZERO); + assertThat(assertions.query("VALUES IF(0 / 0 > 0, true, false)")) + .failure().hasErrorCode(DIVISION_BY_ZERO); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameGroups.java b/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameGroups.java index e2af17ca25153..3c52865168b26 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameGroups.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameGroups.java @@ -23,7 +23,6 @@ import static java.lang.String.format; import static java.math.BigInteger.ONE; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -404,47 +403,47 @@ public void testAllPartitionSameValues() @Test public void testInvalidOffset() { - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC GROUPS x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC GROUPS x PRECEDING) " + "FROM (VALUES (1, 1), (2, -2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 1), (2, -2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS x PRECEDING) " + "FROM (VALUES (1, 1), (2, -2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 1), (2, -2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS x PRECEDING) " + "FROM (VALUES (1, 1), (2, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 1), (2, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); // fail if offset is invalid for null sort key - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 1), (null, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC GROUPS BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 1), (null, -1)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); // test invalid offset of different types - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a GROUPS x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a GROUPS x PRECEDING) " + "FROM (VALUES (1, BIGINT '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a GROUPS x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a GROUPS x PRECEDING) " + "FROM (VALUES (1, INTEGER '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameRange.java b/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameRange.java index 5b3e81a5b9e9d..0735e5156b924 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameRange.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestWindowFrameRange.java @@ -19,7 +19,6 @@ import org.junit.jupiter.api.parallel.Execution; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -566,91 +565,91 @@ public void testNonConstantOffset() @Test public void testInvalidOffset() { - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC RANGE x PRECEDING) " + "FROM (VALUES (1, 0.1), (2, -0.2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a ASC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 0.1), (2, -0.2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE x PRECEDING) " + "FROM (VALUES (1, 0.1), (2, -0.2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 0.1), (2, -0.2)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE x PRECEDING) " + "FROM (VALUES (1, 0.1), (2, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 0.1), (2, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); // fail if offset is invalid for null sort key - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 0.1), (null, null)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a DESC RANGE BETWEEN 1 PRECEDING AND x FOLLOWING) " + "FROM (VALUES (1, 0.1), (null, -0.1)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); // test invalid offset of different types - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (1, BIGINT '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (1, INTEGER '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (SMALLINT '1', SMALLINT '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (TINYINT '1', TINYINT '-1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (1, -1.1e0)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (1, REAL '-1.1')) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (1, -1.0001)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' YEAR)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' MONTH)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' DAY)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' HOUR)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' MINUTE)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); - assertThatThrownBy(() -> assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + + assertThat(assertions.query("SELECT array_agg(a) OVER(ORDER BY a RANGE x PRECEDING) " + "FROM (VALUES (DATE '2001-01-31', INTERVAL '-1' SECOND)) t(a, x)")) - .hasMessage("Window frame offset value must not be negative or null"); + .failure().hasMessage("Window frame offset value must not be negative or null"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestWith.java b/core/trino-main/src/test/java/io/trino/sql/query/TestWith.java index c15ddcbbe3bf5..2d7b6bc94cfa4 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestWith.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestWith.java @@ -29,7 +29,6 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -85,8 +84,8 @@ public void testWithHiddenFields() .matches("SELECT * FROM nation"); // try access hidden column - assertThatThrownBy(() -> assertions.query("WITH t AS (TABLE nation) " + + assertThat(assertions.query("WITH t AS (TABLE nation) " + "SELECT min(row_number) FROM t")) - .hasMessage("line 1:37: Column 'row_number' cannot be resolved"); + .failure().hasMessage("line 1:37: Column 'row_number' cannot be resolved"); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java index c72cf3b47f49c..de7973e3bd1a0 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java @@ -239,6 +239,7 @@ public JdbcTableHandle getTableHandle(ConnectorSession session, PreparedQuery pr PreparedStatement preparedStatement = queryBuilder.prepareStatement(this, session, connection, preparedQuery, Optional.empty())) { ResultSetMetaData metadata = preparedStatement.getMetaData(); if (metadata == null) { + // TODO should be TrinoException throw new UnsupportedOperationException("Query not supported: ResultSetMetaData not available for query: " + preparedQuery.getQuery()); } return new JdbcTableHandle( 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 cae41fe4c3e8c..8e9f172b4668b 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 @@ -1080,12 +1080,12 @@ public void testArithmeticPredicatePushdown() .isFullyPushedDown() .matches("VALUES (BIGINT '3', CAST('CANADA' AS varchar(25)), BIGINT '1')"); - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % 0 = 2")) - .hasMessageContaining("by zero"); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % 0 = 2")) + .failure().hasMessageContaining("by zero"); // Expression that evaluates to 0 for some rows on RHS of modulus - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % (regionkey - 1) = 2")) - .hasMessageContaining("by zero"); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % (regionkey - 1) = 2")) + .failure().hasMessageContaining("by zero"); // TODO add coverage for other arithmetic pushdowns https://github.com/trinodb/trino/issues/14808 } @@ -1988,8 +1988,9 @@ public void testNativeQuerySelectUnsupportedType() // Check that column 'two' is not supported. assertQuery("SELECT column_name FROM information_schema.columns WHERE table_name = '" + unqualifiedTableName + "'", "VALUES 'one', 'three'"); assertUpdate("INSERT INTO " + testTable.getName() + " (one, three) VALUES (123, 'test')", 1); - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'SELECT * FROM %s'))", testTable.getName()))) - .hasMessageContaining("Unsupported type"); + assertThat(query(format("SELECT * FROM TABLE(system.query(query => 'SELECT * FROM %s'))", testTable.getName()))) + // TODO should be TrinoException + .nonTrinoExceptionFailure().hasMessageContaining("Unsupported type"); } } @@ -1998,8 +1999,9 @@ public void testNativeQueryCreateStatement() { skipTestUnless(hasBehavior(SUPPORTS_NATIVE_QUERY)); assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) - .hasMessageContaining("Query not supported: ResultSetMetaData not available for query: CREATE TABLE numbers(n INTEGER)"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) + // TODO should be TrinoException + .nonTrinoExceptionFailure().hasMessageContaining("Query not supported: ResultSetMetaData not available for query: CREATE TABLE numbers(n INTEGER)"); assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); } @@ -2008,8 +2010,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() { skipTestUnless(hasBehavior(SUPPORTS_NATIVE_QUERY)); assertThat(getQueryRunner().tableExists(getSession(), "non_existent_table")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) - .hasMessageContaining("Failed to get table handle for prepared query"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) + .failure().hasMessageContaining("Failed to get table handle for prepared query"); } @Test @@ -2017,8 +2019,9 @@ public void testNativeQueryInsertStatementTableExists() { skipTestUnless(hasBehavior(SUPPORTS_NATIVE_QUERY)); try (TestTable testTable = simpleTable()) { - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) - .hasMessageContaining(format("Query not supported: ResultSetMetaData not available for query: INSERT INTO %s VALUES (3)", testTable.getName())); + assertThat(query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) + // TODO should be TrinoException + .nonTrinoExceptionFailure().hasMessageContaining(format("Query not supported: ResultSetMetaData not available for query: INSERT INTO %s VALUES (3)", testTable.getName())); assertQuery("SELECT * FROM " + testTable.getName(), "VALUES 1, 2"); } } @@ -2027,8 +2030,8 @@ public void testNativeQueryInsertStatementTableExists() public void testNativeQueryIncorrectSyntax() { skipTestUnless(hasBehavior(SUPPORTS_NATIVE_QUERY)); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) - .hasMessageContaining("Failed to get table handle for prepared query"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + .failure().hasMessageContaining("Failed to get table handle for prepared query"); } protected TestTable simpleTable() 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 96f3ffe23d6ba..3756fe786a6d7 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 @@ -608,8 +608,8 @@ public void testDateYearOfEraPredicate() { // Override because the connector throws an exception instead of an empty result when the value is out of supported range assertQuery("SELECT orderdate FROM orders WHERE orderdate = DATE '1997-09-14'", "VALUES DATE '1997-09-14'"); - assertThatThrownBy(() -> query("SELECT * FROM orders WHERE orderdate = DATE '-1996-09-14'")) - .hasMessageMatching(".*Could not cast literal \"-1996-09-14\" to type DATE.*"); + assertThat(query("SELECT * FROM orders WHERE orderdate = DATE '-1996-09-14'")) + .nonTrinoExceptionFailure().hasMessageMatching(".*Could not cast literal \"-1996-09-14\" to type DATE.*"); } @Test @@ -734,8 +734,9 @@ public void testQueryCache() onBigQuery("CREATE MATERIALIZED VIEW test." + materializedView + " AS SELECT count(1) AS cnt FROM tpch.region"); // Verify query cache is empty - assertThatThrownBy(() -> query(createNeverDisposition, "SELECT * FROM test." + materializedView)) - .hasMessageContaining("Not found"); + assertThat(query(createNeverDisposition, "SELECT * FROM test." + materializedView)) + // TODO should be TrinoException, provide a better error message + .nonTrinoExceptionFailure().hasMessageContaining("Not found"); // Populate cache and verify it assertQuery(queryResultsCacheSession, "SELECT * FROM test." + materializedView, "VALUES 5"); assertQuery(createNeverDisposition, "SELECT * FROM test." + materializedView, "VALUES 5"); @@ -786,8 +787,9 @@ public void testWildcardTableWithDifferentColumnDefinition() assertQuery("DESCRIBE test.\"" + wildcardTable + "\"", "VALUES ('value', 'varchar', '', '')"); - assertThatThrownBy(() -> query("SELECT * FROM test.\"" + wildcardTable + "\"")) - .hasMessageContaining("Cannot read field of type INT64 as STRING Field: value"); + assertThat(query("SELECT * FROM test.\"" + wildcardTable + "\"")) + // TODO should be TrinoException + .nonTrinoExceptionFailure().hasMessageContaining("Cannot read field of type INT64 as STRING Field: value"); } finally { onBigQuery("DROP TABLE IF EXISTS test." + firstTable); @@ -798,8 +800,8 @@ public void testWildcardTableWithDifferentColumnDefinition() @Test public void testMissingWildcardTable() { - assertThatThrownBy(() -> query("SELECT * FROM test.\"test_missing_wildcard_table_*\"")) - .hasMessageEndingWith("does not match any table."); + assertThat(query("SELECT * FROM test.\"test_missing_wildcard_table_*\"")) + .nonTrinoExceptionFailure().hasMessageEndingWith("does not match any table."); } @Override @@ -937,8 +939,8 @@ public void testNativeQuerySelectUnsupportedType() onBigQuery("CREATE TABLE test." + tableName + "(one BIGINT, two BIGNUMERIC(40,2), three STRING)"); // Check that column 'two' is not supported. assertQuery("SELECT column_name FROM information_schema.columns WHERE table_schema = 'test' AND table_name = '" + tableName + "'", "VALUES 'one', 'three'"); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(bigquery.system.query(query => 'SELECT * FROM test." + tableName + "'))")) - .hasMessageContaining("Unsupported type"); + assertThat(query("SELECT * FROM TABLE(bigquery.system.query(query => 'SELECT * FROM test." + tableName + "'))")) + .nonTrinoExceptionFailure().hasMessageContaining("Unsupported type"); } finally { onBigQuery("DROP TABLE IF EXISTS test." + tableName); @@ -950,8 +952,8 @@ public void testNativeQueryCreateStatement() { String tableName = "test_create" + randomNameSuffix(); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(bigquery.system.query(query => 'CREATE TABLE test." + tableName + "(n INTEGER)'))")) - .hasMessage("Unsupported statement type: CREATE_TABLE"); + assertThat(query("SELECT * FROM TABLE(bigquery.system.query(query => 'CREATE TABLE test." + tableName + "(n INTEGER)'))")) + .failure().hasMessage("Unsupported statement type: CREATE_TABLE"); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); } @@ -960,7 +962,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() { String tableName = "test_insert" + randomNameSuffix(); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(bigquery.system.query(query => 'INSERT INTO test." + tableName + " VALUES (1)'))")) + assertThat(query("SELECT * FROM TABLE(bigquery.system.query(query => 'INSERT INTO test." + tableName + " VALUES (1)'))")) + .failure() .hasMessageContaining("Failed to get schema for query") .hasStackTraceContaining("%s was not found", tableName); } @@ -971,8 +974,8 @@ public void testNativeQueryInsertStatementTableExists() String tableName = "test_insert" + randomNameSuffix(); try { onBigQuery("CREATE TABLE test." + tableName + "(col BIGINT)"); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(bigquery.system.query(query => 'INSERT INTO test." + tableName + " VALUES (3)'))")) - .hasMessage("Unsupported statement type: INSERT"); + assertThat(query("SELECT * FROM TABLE(bigquery.system.query(query => 'INSERT INTO test." + tableName + " VALUES (3)'))")) + .failure().hasMessage("Unsupported statement type: INSERT"); } finally { onBigQuery("DROP TABLE IF EXISTS test." + tableName); @@ -982,8 +985,8 @@ public void testNativeQueryInsertStatementTableExists() @Test public void testNativeQueryIncorrectSyntax() { - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) - .hasMessageContaining("Failed to get schema for query"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + .failure().hasMessageContaining("Failed to get schema for query"); } @Test diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryAvroConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryAvroConnectorTest.java index 99d74666488d9..ff04e3d227641 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryAvroConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryAvroConnectorTest.java @@ -23,7 +23,6 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestBigQueryAvroConnectorTest extends BaseBigQueryConnectorTest @@ -68,8 +67,9 @@ public void testSelectFailsForColumnName() try { assertUpdate("INSERT INTO " + tableName + " VALUES ('test value')", 1); // The storage API can't read the table, but query based API can read it - assertThatThrownBy(() -> query("SELECT * FROM " + tableName)) - .cause() + assertThat(query("SELECT * FROM " + tableName)) + // TODO should be TrinoException, provide better error message + .nonTrinoExceptionFailure().cause() .hasMessageMatching(".*(Illegal initial character|Invalid name).*"); assertThat(bigQuerySqlExecutor.executeQuery("SELECT * FROM " + tableName).getValues()) .extracting(field -> field.get(0).getStringValue()) diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnectorTest.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnectorTest.java index 892b7b9ad14bf..c6a9356a753b8 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnectorTest.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnectorTest.java @@ -1435,8 +1435,8 @@ public void testNativeQueryCreateTableFailure() { String tableName = "test_create" + randomNameSuffix(); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(cassandra.system.query(query => 'CREATE TABLE tpch." + tableName + "(col INT PRIMARY KEY)'))")) - .hasMessage("Handle doesn't have columns info"); + assertThat(query("SELECT * FROM TABLE(cassandra.system.query(query => 'CREATE TABLE tpch." + tableName + "(col INT PRIMARY KEY)'))")) + .nonTrinoExceptionFailure().hasMessage("Handle doesn't have columns info"); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); } @@ -1445,7 +1445,8 @@ public void testNativeQueryPreparingStatementFailure() { String tableName = "test_insert" + randomNameSuffix(); assertThat(getQueryRunner().tableExists(getSession(), tableName)).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(cassandra.system.query(query => 'INSERT INTO tpch." + tableName + "(col) VALUES (1)'))")) + assertThat(query("SELECT * FROM TABLE(cassandra.system.query(query => 'INSERT INTO tpch." + tableName + "(col) VALUES (1)'))")) + .failure() .hasMessage("Cannot get column definition") .hasStackTraceContaining("unconfigured table"); } @@ -1460,10 +1461,10 @@ public void testNativeQueryUnsupportedStatement() .row(tableName) .build(), new Duration(1, MINUTES)); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(cassandra.system.query(query => 'INSERT INTO tpch." + tableName + "(col) VALUES (3)'))")) - .hasMessage("Handle doesn't have columns info"); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(cassandra.system.query(query => 'DELETE FROM tpch." + tableName + " WHERE col = 1'))")) - .hasMessage("Handle doesn't have columns info"); + assertThat(query("SELECT * FROM TABLE(cassandra.system.query(query => 'INSERT INTO tpch." + tableName + "(col) VALUES (3)'))")) + .nonTrinoExceptionFailure().hasMessage("Handle doesn't have columns info"); + assertThat(query("SELECT * FROM TABLE(cassandra.system.query(query => 'DELETE FROM tpch." + tableName + " WHERE col = 1'))")) + .nonTrinoExceptionFailure().hasMessage("Handle doesn't have columns info"); assertQuery("SELECT * FROM " + tableName, "VALUES 1"); @@ -1473,7 +1474,8 @@ public void testNativeQueryUnsupportedStatement() @Test public void testNativeQueryIncorrectSyntax() { - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + assertThat(query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + .failure() .hasMessage("Cannot get column definition") .hasStackTraceContaining("no viable alternative at input 'some'"); } 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 839f9909319b1..11d1357fab73e 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 @@ -715,8 +715,8 @@ public void testCreateTableWithLongTableName() .hasMessageMatching("(?s).*(Bad path syntax|File name too long).*"); String invalidTableName = baseTableName + "z".repeat(maxTableNameLength().orElseThrow() - baseTableName.length() + 1); - assertThatThrownBy(() -> query("CREATE TABLE " + invalidTableName + " (a bigint)")) - .hasMessageMatching("(?s).*(Cannot open file|File name too long).*"); + assertThat(query("CREATE TABLE " + invalidTableName + " (a bigint)")) + .failure().hasMessageMatching("(?s).*(Cannot open file|File name too long).*"); // ClickHouse lefts a table even if the above statement failed assertThat(getQueryRunner().tableExists(getSession(), validTableName)).isTrue(); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java index e5989df3caa44..e5b5903649a47 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java @@ -649,13 +649,13 @@ public void testCleanupForFailedCreateTableAs() assertThat(getTableFiles(controlTableName)).isNotEmpty(); String tableName = "test_cleanup_for_failed_create_table_as_" + randomNameSuffix(); - assertThatThrownBy(() -> query( + assertThat(query( format("CREATE TABLE " + tableName + " WITH (location = '%s') AS " + "SELECT nationkey from tpch.sf1.nation " + // writer for this part finishes quickly "UNION ALL " + "SELECT 10/(max(orderkey)-max(orderkey)) from tpch.sf10.orders", // writer takes longer to complete and fails at the end getLocationForTable(bucketName, tableName)))) - .hasMessageContaining("Division by zero"); + .failure().hasMessageContaining("Division by zero"); assertEventually(new Duration(5, SECONDS), () -> assertThat(getTableFiles(tableName)).isEmpty()); } @@ -663,13 +663,13 @@ public void testCleanupForFailedCreateTableAs() public void testCleanupForFailedPartitionedCreateTableAs() { String tableName = "test_cleanup_for_failed_partitioned_create_table_as_" + randomNameSuffix(); - assertThatThrownBy(() -> query( + assertThat(query( format("CREATE TABLE " + tableName + "(a, b) WITH (location = '%s', partitioned_by = ARRAY['b']) AS " + "SELECT nationkey, regionkey from tpch.sf1.nation " + // writer for this part finishes quickly "UNION ALL " + "SELECT 10/(max(orderkey)-max(orderkey)), orderkey %% 5 from tpch.sf10.orders group by orderkey %% 5", // writer takes longer to complete and fails at the end getLocationForTable(bucketName, tableName)))) - .hasMessageContaining("Division by zero"); + .failure().hasMessageContaining("Division by zero"); assertEventually(new Duration(5, SECONDS), () -> assertThat(getTableFiles(tableName)).isEmpty()); } @@ -690,7 +690,7 @@ public void testCreateTableAsExistingLocation() assertThat(getTableFiles(tableName)).as("remaining table files").isNotEmpty(); // crate with non-empty target directory should fail - assertThatThrownBy(() -> query(createTableStatement)).hasMessageContaining("Target location cannot contain any files"); + assertThat(query(createTableStatement)).failure().hasMessageContaining("Target location cannot contain any files"); } @Test 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 7fd2a9bbdce14..543187b98f90a 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 @@ -1163,15 +1163,15 @@ private void testTableWithNonNullableColumns(ColumnMappingMode mode) assertUpdate("CREATE TABLE " + tableName + "(col1 INTEGER NOT NULL, col2 INTEGER, col3 INTEGER) WITH (column_mapping_mode='" + mode + "')"); assertUpdate("INSERT INTO " + tableName + " VALUES(1, 10, 100)", 1); assertUpdate("INSERT INTO " + tableName + " VALUES(2, 20, 200)", 1); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES(null, 30, 300)")) - .hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES(TRY(5/0), 40, 400)")) - .hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); - - assertThatThrownBy(() -> query("UPDATE " + tableName + " SET col1 = NULL where col3 = 100")) - .hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); - assertThatThrownBy(() -> query("UPDATE " + tableName + " SET col1 = TRY(5/0) where col3 = 200")) - .hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); + assertThat(query("INSERT INTO " + tableName + " VALUES(null, 30, 300)")) + .failure().hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); + assertThat(query("INSERT INTO " + tableName + " VALUES(TRY(5/0), 40, 400)")) + .failure().hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); + + assertThat(query("UPDATE " + tableName + " SET col1 = NULL where col3 = 100")) + .failure().hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); + assertThat(query("UPDATE " + tableName + " SET col1 = TRY(5/0) where col3 = 200")) + .failure().hasMessageContaining("NULL value not allowed for NOT NULL column: col1"); assertQuery("SELECT * FROM " + tableName, "VALUES(1, 10, 100), (2, 20, 200)"); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndHmsConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndHmsConnectorSmokeTest.java index d5c83ef50c1a0..f646055b288fe 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndHmsConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndHmsConnectorSmokeTest.java @@ -189,10 +189,10 @@ public void testDeltaColumnInvariant() assertUpdate("INSERT INTO " + tableName + " VALUES(2)", 1); assertQuery("SELECT * FROM " + tableName, "VALUES (1), (2)"); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES(3)")) - .hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); - assertThatThrownBy(() -> query("UPDATE " + tableName + " SET dummy = 3 WHERE dummy = 1")) - .hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); + assertThat(query("INSERT INTO " + tableName + " VALUES(3)")) + .failure().hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); + assertThat(query("UPDATE " + tableName + " SET dummy = 3 WHERE dummy = 1")) + .failure().hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); assertQuery("SELECT * FROM " + tableName, "VALUES (1), (2)"); } @@ -211,10 +211,10 @@ public void testDeltaColumnInvariantWriterFeature() assertUpdate("INSERT INTO " + tableName + " VALUES 2", 1); assertQuery("SELECT * FROM " + tableName, "VALUES 1, 2"); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES 3")) - .hasMessageContaining("Check constraint violation: (\"col_invariants\" < 3)"); - assertThatThrownBy(() -> query("UPDATE " + tableName + " SET col_invariants = 3 WHERE col_invariants = 1")) - .hasMessageContaining("Check constraint violation: (\"col_invariants\" < 3)"); + assertThat(query("INSERT INTO " + tableName + " VALUES 3")) + .failure().hasMessageContaining("Check constraint violation: (\"col_invariants\" < 3)"); + assertThat(query("UPDATE " + tableName + " SET col_invariants = 3 WHERE col_invariants = 1")) + .failure().hasMessageContaining("Check constraint violation: (\"col_invariants\" < 3)"); assertQuery("SELECT * FROM " + tableName, "VALUES 1, 2"); } @@ -230,15 +230,15 @@ public void testSchemaEvolutionOnTableWithColumnInvariant() tableName, getLocationForTable(bucketName, tableName))); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES(3)")) - .hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); + assertThat(query("INSERT INTO " + tableName + " VALUES(3)")) + .failure().hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN c INT"); assertUpdate("COMMENT ON COLUMN " + tableName + ".c IS 'example column comment'"); assertUpdate("COMMENT ON TABLE " + tableName + " IS 'example table comment'"); - assertThatThrownBy(() -> query("INSERT INTO " + tableName + " VALUES(3, 30)")) - .hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); + assertThat(query("INSERT INTO " + tableName + " VALUES(3, 30)")) + .failure().hasMessageContaining("Check constraint violation: (\"dummy\" < 3)"); assertUpdate("INSERT INTO " + tableName + " VALUES(2, 20)", 1); assertQuery("SELECT * FROM " + tableName, "VALUES (1, NULL), (2, 20)"); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java index e84783eb94a81..8001c4eae52df 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java @@ -30,7 +30,6 @@ import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -111,16 +110,16 @@ public void testShowTables() assertQuery("SHOW TABLES FROM delta." + schema, "VALUES 'hive_table', 'hive_view', 'delta_table'"); assertQuery("SHOW TABLES FROM hive." + schema, "VALUES 'hive_table', 'hive_view', 'delta_table'"); - assertThatThrownBy(() -> query("SHOW CREATE TABLE delta." + schema + ".hive_table")) - .hasMessageContaining("not a Delta Lake table"); - assertThatThrownBy(() -> query("SHOW CREATE TABLE delta." + schema + ".hive_view")) - .hasMessageContaining("not a Delta Lake table"); - assertThatThrownBy(() -> query("SHOW CREATE TABLE hive." + schema + ".delta_table")) - .hasMessageContaining("Cannot query Delta Lake table"); + assertThat(query("SHOW CREATE TABLE delta." + schema + ".hive_table")) + .failure().hasMessageContaining("not a Delta Lake table"); + assertThat(query("SHOW CREATE TABLE delta." + schema + ".hive_view")) + .failure().hasMessageContaining("not a Delta Lake table"); + assertThat(query("SHOW CREATE TABLE hive." + schema + ".delta_table")) + .failure().hasMessageContaining("Cannot query Delta Lake table"); - assertThatThrownBy(() -> query("DESCRIBE delta." + schema + ".hive_table")) - .hasMessageContaining("not a Delta Lake table"); - assertThatThrownBy(() -> query("DESCRIBE hive." + schema + ".delta_table")) - .hasMessageContaining("Cannot query Delta Lake table"); + assertThat(query("DESCRIBE delta." + schema + ".hive_table")) + .failure().hasMessageContaining("not a Delta Lake table"); + assertThat(query("DESCRIBE hive." + schema + ".delta_table")) + .failure().hasMessageContaining("Cannot query Delta Lake table"); } } diff --git a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidConnectorTest.java b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidConnectorTest.java index af199ddd910c7..afb7dec11a827 100644 --- a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidConnectorTest.java +++ b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidConnectorTest.java @@ -337,8 +337,8 @@ public void testNativeQueryCreateStatement() { // override because Druid fails to prepare statement, while other connectors succeed in preparing statement and then fail because of no metadata available assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) - .hasMessageContaining("Failed to get table handle for prepared query"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) + .failure().hasMessageContaining("Failed to get table handle for prepared query"); assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); } diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java index 07abe0792fcd6..27cf1d0454756 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java @@ -1887,12 +1887,12 @@ public void testQueryTableFunction() "VALUES '[]'"); // syntax error - assertThatThrownBy(() -> query("SELECT * " + + assertThat(query("SELECT * " + format("FROM TABLE(%s.system.raw_query(", catalogName) + "schema => 'tpch', " + "index => 'nation', " + "query => 'wrong syntax')) t(result)")) - .hasMessageContaining("json_parse_exception"); + .failure().hasMessageContaining("json_parse_exception"); } protected void assertTableDoesNotExist(String name) diff --git a/plugin/trino-google-sheets/pom.xml b/plugin/trino-google-sheets/pom.xml index a1d32e594c363..434636984248f 100644 --- a/plugin/trino-google-sheets/pom.xml +++ b/plugin/trino-google-sheets/pom.xml @@ -185,6 +185,13 @@ test + + io.trino + trino-main + test-jar + test + + io.trino trino-testing diff --git a/plugin/trino-google-sheets/src/test/java/io/trino/plugin/google/sheets/TestGoogleSheets.java b/plugin/trino-google-sheets/src/test/java/io/trino/plugin/google/sheets/TestGoogleSheets.java index d441a98fd2bdb..1514ffac4a66d 100644 --- a/plugin/trino-google-sheets/src/test/java/io/trino/plugin/google/sheets/TestGoogleSheets.java +++ b/plugin/trino-google-sheets/src/test/java/io/trino/plugin/google/sheets/TestGoogleSheets.java @@ -42,7 +42,6 @@ import static io.trino.testing.assertions.Assert.assertEventually; import static java.lang.Math.toIntExact; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestGoogleSheets extends AbstractTestQueryFramework @@ -266,33 +265,33 @@ public void testSheetQueryWithSheetAndRowAndColumnRange() public void testSheetQueryWithSheetRangeInIdFails() { // Sheet ids with "#" are explicitly forbidden since "#" is the sheet separator - assertThatThrownBy(() -> query( + assertThat(query( "SELECT * FROM TABLE(gsheets.system.sheet(id => '%s#%s'))".formatted(DATA_SHEET_ID, "number_text"))) - .hasMessageContaining("Google sheet ID %s cannot contain '#'. Provide a range through the 'range' argument.".formatted(DATA_SHEET_ID + "#number_text")); + .failure().hasMessageContaining("Google sheet ID %s cannot contain '#'. Provide a range through the 'range' argument.".formatted(DATA_SHEET_ID + "#number_text")); // Attempting to put a sheet range in the id fails since the sheet id is invalid - assertThatThrownBy(() -> query( + assertThat(query( "SELECT * FROM TABLE(gsheets.system.sheet(id => '%s%s'))".formatted(DATA_SHEET_ID, "number_text"))) - .hasMessageContaining("Failed reading data from sheet: %snumber_text#$1:$10000".formatted(DATA_SHEET_ID)); + .failure().hasMessageContaining("Failed reading data from sheet: %snumber_text#$1:$10000".formatted(DATA_SHEET_ID)); } @Test public void testSheetQueryWithNoDataInRangeFails() { - assertThatThrownBy(() -> query( + assertThat(query( "SELECT * FROM TABLE(gsheets.system.sheet(id => '%s', range => '%s'))".formatted(DATA_SHEET_ID, "number_text!D1:D1"))) - .hasMessageContaining("No non-empty cells found in sheet: %s#number_text!D1:D1".formatted(DATA_SHEET_ID)); + .failure().hasMessageContaining("No non-empty cells found in sheet: %s#number_text!D1:D1".formatted(DATA_SHEET_ID)); - assertThatThrownBy(() -> query( + assertThat(query( "SELECT * FROM TABLE(gsheets.system.sheet(id => '%s', range => '%s'))".formatted(DATA_SHEET_ID, "number_text!D12:E13"))) - .hasMessageContaining("No non-empty cells found in sheet: %s#number_text!D12:E13".formatted(DATA_SHEET_ID)); + .failure().hasMessageContaining("No non-empty cells found in sheet: %s#number_text!D12:E13".formatted(DATA_SHEET_ID)); } @Test public void testSheetQueryWithInvalidSheetId() { - assertThatThrownBy(() -> query("SELECT * FROM TABLE(gsheets.system.sheet(id => 'DOESNOTEXIST'))")) - .hasMessageContaining("Failed reading data from sheet: DOESNOTEXIST"); + assertThat(query("SELECT * FROM TABLE(gsheets.system.sheet(id => 'DOESNOTEXIST'))")) + .failure().hasMessageContaining("Failed reading data from sheet: DOESNOTEXIST"); } @Test 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 ba60c3215cd41..36abe8bade5c9 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 @@ -3587,10 +3587,10 @@ public void testPartitionPerScanLimitWithMultiplePartitionColumns() .matches("VALUES BIGINT '1000'"); // verify cannot query more than 1000 partitions - assertThatThrownBy(() -> query("SELECT count(*) FROM " + tableName + " WHERE part1 IS NULL AND part2 <= 1001")) - .hasMessage("Query over table 'tpch.%s' can potentially read more than 1000 partitions", tableName); - assertThatThrownBy(() -> query("SELECT count(*) FROM " + tableName)) - .hasMessage("Query over table 'tpch.%s' can potentially read more than 1000 partitions", tableName); + assertThat(query("SELECT count(*) FROM " + tableName + " WHERE part1 IS NULL AND part2 <= 1001")) + .failure().hasMessage("Query over table 'tpch.%s' can potentially read more than 1000 partitions", tableName); + assertThat(query("SELECT count(*) FROM " + tableName)) + .failure().hasMessage("Query over table 'tpch.%s' can potentially read more than 1000 partitions", tableName); // verify we can query with a predicate that is not representable as a TupleDomain assertThat(query("SELECT * FROM " + tableName + " WHERE part1 % 400 = 3")) // may be translated to Domain.all @@ -8764,8 +8764,8 @@ private void testHiddenColumnNameConflict(String columnName) getQueryRunner()::execute, "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))) { - assertThatThrownBy(() -> query("SELECT * FROM " + table.getName())) - .hasMessageContaining("Multiple entries with same key: " + columnName); + assertThat(query("SELECT * FROM " + table.getName())) + .nonTrinoExceptionFailure().hasMessageContaining("Multiple entries with same key: " + columnName); } } @@ -9004,9 +9004,8 @@ public void testCollidingMixedCaseProperty() assertUpdate("CREATE TABLE %s (c1 integer) WITH (extra_properties = MAP(ARRAY['one', 'ONE'], ARRAY['one', 'ONE']))".formatted(tableName)); // TODO: (https://github.com/trinodb/trino/issues/17) This should run successfully - assertThatThrownBy(() -> query("SELECT * FROM \"%s$properties\"".formatted(tableName))) - .isInstanceOf(QueryFailedException.class) - .hasMessageContaining("Multiple entries with same key: one=one and one=one"); + assertThat(query("SELECT * FROM \"%s$properties\"".formatted(tableName))) + .nonTrinoExceptionFailure().hasMessageContaining("Multiple entries with same key: one=one and one=one"); assertUpdate("DROP TABLE %s".formatted(tableName)); } @@ -9029,7 +9028,8 @@ public void testSelectWithShortZoneId() getQueryRunner()::execute, "test_select_with_short_zone_id_", "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(tempDir))) { - assertThatThrownBy(() -> query("SELECT * FROM %s".formatted(testTable.getName()))) + assertThat(query("SELECT * FROM %s".formatted(testTable.getName()))) + .failure() .hasMessageMatching(".*Failed to read ORC file: .*") .hasStackTraceContaining("Unknown time-zone ID: EST"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java index 869f092e1184a..fd2ccbf226c2d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java @@ -1826,8 +1826,8 @@ public void testDropStatsPartitionedTable() ")").formatted(getFullyQualifiedTestTableName(tableName))); // Drop stats for partition which does not exist - assertThatThrownBy(() -> query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['partnotfound', '999']])", HIVE_TEST_SCHEMA, tableName))) - .hasMessage("No partition found for name: p_varchar=partnotfound/p_integer=999"); + assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['partnotfound', '999']])", HIVE_TEST_SCHEMA, tableName))) + .failure().hasMessage("No partition found for name: p_varchar=partnotfound/p_integer=999"); assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES (1, 'part1', 10) , (2, 'part2', 10), (12, 'part2', 20)", 3); @@ -1857,8 +1857,8 @@ public void testDropStatsPartitionedTable() assertUpdate("DELETE FROM " + getFullyQualifiedTestTableName(tableName) + " WHERE p_varchar ='part1' and p_integer = 10"); // Drop stats for partition which does not exist - assertThatThrownBy(() -> query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName))) - .hasMessage("No partition found for name: p_varchar=part1/p_integer=10"); + assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName))) + .failure().hasMessage("No partition found for name: p_varchar=part1/p_integer=10"); assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), """ @@ -1906,6 +1906,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name + "(x double) RETURNS double COMMENT 't88' RETURN x * 8.8"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") @@ -1924,6 +1925,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name2 + "(s varchar) RETURNS varchar RETURN 'Hello ' || s"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") @@ -2019,6 +2021,7 @@ protected void assertOverwritePartition(String testTable) ImmutableList.of("'CZECH'", "'Test Data'", "26", "5")))); query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) .assertThat() + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row("POLAND", "Test Data", 25L, 5L) @@ -2031,6 +2034,7 @@ protected void assertOverwritePartition(String testTable) ImmutableList.of("'POLAND'", "'Overwrite'", "25", "5")))); query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) .assertThat() + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row("POLAND", "Overwrite", 25L, 5L) @@ -2098,12 +2102,14 @@ private void testWriteWithFileSize(String testTable, int scaleFactorInThousands, computeActual(format("INSERT INTO " + testTable + " SELECT %s, %s, regionkey FROM tpch.tiny.nation WHERE nationkey = 9", scaledColumnExpression, scaledColumnExpression)); query(format("SELECT length(col1) FROM %s", testTable)) .assertThat() + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(114L * scaleFactorInThousands * 1000) .build()); query(format("SELECT \"$file_size\" BETWEEN %d AND %d FROM %s", fileSizeRangeStart, fileSizeRangeEnd, testTable)) .assertThat() + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(true) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java index bb70593a25585..0b6b751c0f0b9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java @@ -24,7 +24,6 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static java.util.concurrent.TimeUnit.MINUTES; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestHiveAnalyzeCorruptStatistics extends AbstractTestQueryFramework @@ -56,12 +55,13 @@ public void testAnalyzeCorruptColumnStatisticsOnEmptyTable() assertQuerySucceeds("SHOW STATS FOR " + tableName); // ANALYZE and drop_stats are unsupported for tables having broken column statistics - assertThatThrownBy(() -> query("ANALYZE " + tableName)) + assertThat(query("ANALYZE " + tableName)) + .failure() .hasMessage("Unexpected 2 statistics for 1 columns") .hasStackTraceContaining("ThriftHiveMetastore.setTableColumnStatistics"); - assertThatThrownBy(() -> query("CALL system.drop_stats('tpch', '" + tableName + "')")) - .hasMessageContaining("The query returned more than one instance BUT either unique is set to true or only aggregates are to be returned, so should have returned one result maximum"); + assertThat(query("CALL system.drop_stats('tpch', '" + tableName + "')")) + .failure().hasMessageContaining("The query returned more than one instance BUT either unique is set to true or only aggregates are to be returned, so should have returned one result maximum"); assertUpdate("DROP TABLE " + tableName); } @@ -115,8 +115,8 @@ public void testAnalyzeCorruptPartitionStatisticsOnEmptyTable() // ANALYZE succeeds for a partitioned table with corrupt stats unlike a non-partitioned table with corrupt stats assertUpdate("ANALYZE " + tableName, 1); - assertThatThrownBy(() -> query("CALL system.drop_stats('tpch', '" + tableName + "')")) - .hasMessageContaining("The query returned more than one instance BUT either unique is set to true or only aggregates are to be returned, so should have returned one result maximum"); + assertThat(query("CALL system.drop_stats('tpch', '" + tableName + "')")) + .failure().hasMessageContaining("The query returned more than one instance BUT either unique is set to true or only aggregates are to be returned, so should have returned one result maximum"); assertUpdate("DROP TABLE " + tableName); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java index 0487184e504fa..58c7971aef1c7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java @@ -325,6 +325,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name + "(x double) RETURNS double COMMENT 't88' RETURN x * 8.8"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") @@ -343,6 +344,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name2 + "(s varchar) RETURNS varchar RETURN 'Hello ' || s"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") 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 b728a95aba9b8..f7a810fec0409 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 @@ -1278,8 +1278,9 @@ public void testCreateSortedTableWithSortTransform() private void testCreateSortedTableWithSortTransform(String columnName, String sortField) { String tableName = "test_sort_with_transform_" + randomNameSuffix(); - assertThatThrownBy(() -> query(format("CREATE TABLE %s (%s TIMESTAMP(6)) WITH (sorted_by = ARRAY['%s'])", tableName, columnName, sortField))) - .hasMessageContaining("Unable to parse sort field"); + assertThat(query(format("CREATE TABLE %s (%s TIMESTAMP(6)) WITH (sorted_by = ARRAY['%s'])", tableName, columnName, sortField))) + // TODO throw TrinoException indicating user error + .nonTrinoExceptionFailure().hasMessageContaining("Unable to parse sort field"); } @Test @@ -1381,15 +1382,17 @@ public void testUpdateWithSortOrder() public void testSortingOnNestedField() { String tableName = "test_sorting_on_nested_field" + randomNameSuffix(); - assertThatThrownBy(() -> query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + + assertThat(query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + "WITH (sorted_by = ARRAY['row_t.comment'])")) - .hasMessageContaining("Unable to parse sort field: [row_t.comment]"); - assertThatThrownBy(() -> query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + + // TODO throw TrinoException indicating user error + .nonTrinoExceptionFailure().hasMessageContaining("Unable to parse sort field: [row_t.comment]"); + assertThat(query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + "WITH (sorted_by = ARRAY['\"row_t\".\"comment\"'])")) - .hasMessageContaining("Unable to parse sort field: [\"row_t\".\"comment\"]"); - assertThatThrownBy(() -> query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + + // TODO throw TrinoException indicating user error + .nonTrinoExceptionFailure().hasMessageContaining("Unable to parse sort field: [\"row_t\".\"comment\"]"); + assertThat(query("CREATE TABLE " + tableName + " (nationkey BIGINT, row_t ROW(name VARCHAR, regionkey BIGINT, comment VARCHAR)) " + "WITH (sorted_by = ARRAY['\"row_t.comment\"'])")) - .hasMessageContaining("Column not found: row_t.comment"); + .failure().hasMessageContaining("Column not found: row_t.comment"); } @Test @@ -1401,8 +1404,8 @@ public void testDroppingSortColumn() "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); - assertThatThrownBy(() -> query("ALTER TABLE " + table.getName() + " DROP COLUMN comment")) - .hasMessageContaining("Cannot find source column for sort field"); + assertThat(query("ALTER TABLE " + table.getName() + " DROP COLUMN comment")) + .failure().hasMessageContaining("Cannot find source column for sort field"); } } @@ -5202,8 +5205,8 @@ public void testOptimizeSnapshot() assertUpdate("CREATE TABLE " + tableName + " (a) AS VALUES 11", 1); long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); - assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE".formatted(tableName, snapshotId))) - .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); + assertThat(query("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE".formatted(tableName, snapshotId))) + .failure().hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5213,10 +5216,10 @@ public void testOptimizeSnapshot() @Test public void testOptimizeSystemTable() { - assertThatThrownBy(() -> query("ALTER TABLE \"nation$files\" EXECUTE OPTIMIZE")) - .hasMessage("This connector does not support table procedures"); - assertThatThrownBy(() -> query("ALTER TABLE \"nation$snapshots\" EXECUTE OPTIMIZE")) - .hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$files\" EXECUTE OPTIMIZE")) + .failure().hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$snapshots\" EXECUTE OPTIMIZE")) + .failure().hasMessage("This connector does not support table procedures"); } private List getActiveFiles(String tableName) @@ -5614,8 +5617,8 @@ public void testExpireSnapshotsOnSnapshot() assertUpdate("CREATE TABLE " + tableName + " (a) AS VALUES 11", 1); long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); - assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE EXPIRE_SNAPSHOTS".formatted(tableName, snapshotId))) - .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); + assertThat(query("ALTER TABLE \"%s@%d\" EXECUTE EXPIRE_SNAPSHOTS".formatted(tableName, snapshotId))) + .failure().hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5625,10 +5628,10 @@ public void testExpireSnapshotsOnSnapshot() @Test public void testExpireSnapshotsSystemTable() { - assertThatThrownBy(() -> query("ALTER TABLE \"nation$files\" EXECUTE EXPIRE_SNAPSHOTS")) - .hasMessage("This connector does not support table procedures"); - assertThatThrownBy(() -> query("ALTER TABLE \"nation$snapshots\" EXECUTE EXPIRE_SNAPSHOTS")) - .hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$files\" EXECUTE EXPIRE_SNAPSHOTS")) + .failure().hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$snapshots\" EXECUTE EXPIRE_SNAPSHOTS")) + .failure().hasMessage("This connector does not support table procedures"); } @Test @@ -5806,8 +5809,8 @@ public void testRemoveOrphanFilesOnSnapshot() assertUpdate("CREATE TABLE " + tableName + " (a) AS VALUES 11", 1); long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); - assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE REMOVE_ORPHAN_FILES".formatted(tableName, snapshotId))) - .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); + assertThat(query("ALTER TABLE \"%s@%d\" EXECUTE REMOVE_ORPHAN_FILES".formatted(tableName, snapshotId))) + .failure().hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5817,10 +5820,10 @@ public void testRemoveOrphanFilesOnSnapshot() @Test public void testRemoveOrphanFilesSystemTable() { - assertThatThrownBy(() -> query("ALTER TABLE \"nation$files\" EXECUTE REMOVE_ORPHAN_FILES")) - .hasMessage("This connector does not support table procedures"); - assertThatThrownBy(() -> query("ALTER TABLE \"nation$snapshots\" EXECUTE REMOVE_ORPHAN_FILES")) - .hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$files\" EXECUTE REMOVE_ORPHAN_FILES")) + .failure().hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$snapshots\" EXECUTE REMOVE_ORPHAN_FILES")) + .failure().hasMessage("This connector does not support table procedures"); } @Test @@ -5863,8 +5866,8 @@ public void testUpdatingInvalidTableProperty() { String tableName = "test_updating_invalid_table_property_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (a INT, b INT)"); - assertThatThrownBy(() -> query("ALTER TABLE " + tableName + " SET PROPERTIES not_a_valid_table_property = 'a value'")) - .hasMessage("Catalog 'iceberg' table property 'not_a_valid_table_property' does not exist"); + assertThat(query("ALTER TABLE " + tableName + " SET PROPERTIES not_a_valid_table_property = 'a value'")) + .failure().hasMessage("Catalog 'iceberg' table property 'not_a_valid_table_property' does not exist"); assertUpdate("DROP TABLE " + tableName); } @@ -5949,20 +5952,20 @@ public void testModifyingOldSnapshotIsNotPossible() long oldSnapshotId = getCurrentSnapshotId(tableName); assertUpdate(format("INSERT INTO %s VALUES 4,5,6", tableName), 3); assertQuery(format("SELECT * FROM %s FOR VERSION AS OF %d", tableName, oldSnapshotId), "VALUES 1,2,3"); - assertThatThrownBy(() -> query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, oldSnapshotId))) - .hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); - assertThatThrownBy(() -> query(format("DELETE FROM \"%s@%d\" WHERE col = 5", tableName, oldSnapshotId))) - .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); - assertThatThrownBy(() -> query(format("UPDATE \"%s@%d\" SET col = 50 WHERE col = 5", tableName, oldSnapshotId))) - .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); - assertThatThrownBy(() -> query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, getCurrentSnapshotId(tableName)))) - .hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); - assertThatThrownBy(() -> query(format("DELETE FROM \"%s@%d\" WHERE col = 9", tableName, getCurrentSnapshotId(tableName)))) - .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); + assertThat(query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, oldSnapshotId))) + .failure().hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); + assertThat(query(format("DELETE FROM \"%s@%d\" WHERE col = 5", tableName, oldSnapshotId))) + .failure().hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); + assertThat(query(format("UPDATE \"%s@%d\" SET col = 50 WHERE col = 5", tableName, oldSnapshotId))) + .failure().hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); + assertThat(query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, getCurrentSnapshotId(tableName)))) + .failure().hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); + assertThat(query(format("DELETE FROM \"%s@%d\" WHERE col = 9", tableName, getCurrentSnapshotId(tableName)))) + .failure().hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); assertThatThrownBy(() -> assertUpdate(format("UPDATE \"%s@%d\" set col = 50 WHERE col = 5", tableName, getCurrentSnapshotId(tableName)))) .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); - assertThatThrownBy(() -> query(format("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE", tableName, oldSnapshotId))) - .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); + assertThat(query(format("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE", tableName, oldSnapshotId))) + .failure().hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); assertQuery(format("SELECT * FROM %s", tableName), "VALUES 1,2,3,4,5,6"); assertUpdate("DROP TABLE " + tableName); @@ -7079,8 +7082,8 @@ public void testNoRetryWhenMetadataFileInvalid() // Corrupt metadata file by overwriting the invalid metadata content outputStream.write(modifiedJson.getBytes(UTF_8)); } - assertThatThrownBy(() -> query("SELECT * FROM " + tableName)) - .hasMessage("Invalid metadata file for table tpch.%s".formatted(tableName)); + assertThat(query("SELECT * FROM " + tableName)) + .failure().hasMessage("Invalid metadata file for table tpch.%s".formatted(tableName)); assertUpdate("DROP TABLE " + tableName); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java index ba1a91c729f7a..cfbacc3017695 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java @@ -542,10 +542,10 @@ public void testCreateMaterializedViewWhenTableExists() { String schema = getSession().getSchema().orElseThrow(); assertUpdate("CREATE TABLE test_create_materialized_view_when_table_exists (a INT, b INT)"); - assertThatThrownBy(() -> query("CREATE OR REPLACE MATERIALIZED VIEW test_create_materialized_view_when_table_exists AS SELECT sum(1) AS num_rows FROM base_table2")) - .hasMessage("Existing table is not a Materialized View: " + schema + ".test_create_materialized_view_when_table_exists"); - assertThatThrownBy(() -> query("CREATE MATERIALIZED VIEW IF NOT EXISTS test_create_materialized_view_when_table_exists AS SELECT sum(1) AS num_rows FROM base_table2")) - .hasMessage("Existing table is not a Materialized View: " + schema + ".test_create_materialized_view_when_table_exists"); + assertThat(query("CREATE OR REPLACE MATERIALIZED VIEW test_create_materialized_view_when_table_exists AS SELECT sum(1) AS num_rows FROM base_table2")) + .failure().hasMessage("Existing table is not a Materialized View: " + schema + ".test_create_materialized_view_when_table_exists"); + assertThat(query("CREATE MATERIALIZED VIEW IF NOT EXISTS test_create_materialized_view_when_table_exists AS SELECT sum(1) AS num_rows FROM base_table2")) + .failure().hasMessage("Existing table is not a Materialized View: " + schema + ".test_create_materialized_view_when_table_exists"); assertUpdate("DROP TABLE test_create_materialized_view_when_table_exists"); } @@ -554,8 +554,8 @@ public void testDropMaterializedViewCannotDropTable() { String schema = getSession().getSchema().orElseThrow(); assertUpdate("CREATE TABLE test_drop_materialized_view_cannot_drop_table (a INT, b INT)"); - assertThatThrownBy(() -> query("DROP MATERIALIZED VIEW test_drop_materialized_view_cannot_drop_table")) - .hasMessageContaining("Materialized view 'iceberg." + schema + ".test_drop_materialized_view_cannot_drop_table' does not exist, but a table with that name exists"); + assertThat(query("DROP MATERIALIZED VIEW test_drop_materialized_view_cannot_drop_table")) + .failure().hasMessageContaining("Materialized view 'iceberg." + schema + ".test_drop_materialized_view_cannot_drop_table' does not exist, but a table with that name exists"); assertUpdate("DROP TABLE test_drop_materialized_view_cannot_drop_table"); } @@ -564,8 +564,8 @@ public void testRenameMaterializedViewCannotRenameTable() { String schema = getSession().getSchema().orElseThrow(); assertUpdate("CREATE TABLE test_rename_materialized_view_cannot_rename_table (a INT, b INT)"); - assertThatThrownBy(() -> query("ALTER MATERIALIZED VIEW test_rename_materialized_view_cannot_rename_table RENAME TO new_materialized_view_name")) - .hasMessageContaining("Materialized View 'iceberg." + schema + ".test_rename_materialized_view_cannot_rename_table' does not exist, but a table with that name exists"); + assertThat(query("ALTER MATERIALIZED VIEW test_rename_materialized_view_cannot_rename_table RENAME TO new_materialized_view_name")) + .failure().hasMessageContaining("Materialized View 'iceberg." + schema + ".test_rename_materialized_view_cannot_rename_table' does not exist, but a table with that name exists"); assertUpdate("DROP TABLE test_rename_materialized_view_cannot_rename_table"); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseSharedMetastoreTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseSharedMetastoreTest.java index ac02760b4f23a..6bc5e190f62e4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseSharedMetastoreTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseSharedMetastoreTest.java @@ -24,7 +24,6 @@ import static java.lang.String.format; import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public abstract class BaseSharedMetastoreTest extends AbstractTestQueryFramework @@ -45,10 +44,10 @@ public void testSelect() assertQuery("SELECT * FROM iceberg_with_redirections." + schema + ".nation", "SELECT * FROM nation"); assertQuery("SELECT * FROM iceberg_with_redirections." + schema + ".region", "SELECT * FROM region"); - assertThatThrownBy(() -> query("SELECT * FROM iceberg." + schema + ".region")) - .hasMessageContaining("Not an Iceberg table"); - assertThatThrownBy(() -> query("SELECT * FROM hive." + schema + ".nation")) - .hasMessageContaining("Cannot query Iceberg table"); + assertThat(query("SELECT * FROM iceberg." + schema + ".region")) + .failure().hasMessageContaining("Not an Iceberg table"); + assertThat(query("SELECT * FROM hive." + schema + ".nation")) + .failure().hasMessageContaining("Cannot query Iceberg table"); } @Test @@ -92,15 +91,15 @@ public void testShowTables() assertQuery("SHOW TABLES FROM hive_with_redirections." + schema, "VALUES 'region', 'nation'"); assertQuery("SHOW TABLES FROM iceberg_with_redirections." + schema, "VALUES 'region', 'nation'"); - assertThatThrownBy(() -> query("SHOW CREATE TABLE iceberg." + schema + ".region")) - .hasMessageContaining("Not an Iceberg table"); - assertThatThrownBy(() -> query("SHOW CREATE TABLE hive." + schema + ".nation")) - .hasMessageContaining("Cannot query Iceberg table"); + assertThat(query("SHOW CREATE TABLE iceberg." + schema + ".region")) + .failure().hasMessageContaining("Not an Iceberg table"); + assertThat(query("SHOW CREATE TABLE hive." + schema + ".nation")) + .failure().hasMessageContaining("Cannot query Iceberg table"); - assertThatThrownBy(() -> query("DESCRIBE iceberg." + schema + ".region")) - .hasMessageContaining("Not an Iceberg table"); - assertThatThrownBy(() -> query("DESCRIBE hive." + schema + ".nation")) - .hasMessageContaining("Cannot query Iceberg table"); + assertThat(query("DESCRIBE iceberg." + schema + ".region")) + .failure().hasMessageContaining("Not an Iceberg table"); + assertThat(query("DESCRIBE hive." + schema + ".nation")) + .failure().hasMessageContaining("Cannot query Iceberg table"); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java index f47cad8988447..d1fbc9a253dbd 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java @@ -32,7 +32,6 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestIcebergMigrateProcedure extends AbstractTestQueryFramework @@ -347,8 +346,8 @@ public void testMigrateUnsupportedTableFormat() assertUpdate("CREATE TABLE " + hiveTableName + " WITH (format = 'RCBINARY') AS SELECT 1 x", 1); - assertThatThrownBy(() -> query("CALL iceberg.system.migrate('tpch', '" + tableName + "')")) - .hasStackTraceContaining("Unsupported storage format: RCBINARY"); + assertThat(query("CALL iceberg.system.migrate('tpch', '" + tableName + "')")) + .failure().hasStackTraceContaining("Unsupported storage format: RCBINARY"); assertQuery("SELECT * FROM " + hiveTableName, "VALUES 1"); assertQueryFails("SELECT * FROM " + icebergTableName, "Not an Iceberg table: .*"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java index 8b2a0632de086..91a27d75b5aa6 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java @@ -38,7 +38,6 @@ import static java.math.RoundingMode.UP; import static java.util.stream.Collectors.joining; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestIcebergStatistics extends AbstractTestQueryFramework @@ -596,8 +595,8 @@ public void testAnalyzeSnapshot() assertUpdate("CREATE TABLE " + tableName + " (a) AS VALUES 11", 1); long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); - assertThatThrownBy(() -> query("ANALYZE \"%s@%d\"".formatted(tableName, snapshotId))) - .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); + assertThat(query("ANALYZE \"%s@%d\"".formatted(tableName, snapshotId))) + .failure().hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -607,12 +606,12 @@ public void testAnalyzeSnapshot() @Test public void testAnalyzeSystemTable() { - assertThatThrownBy(() -> query("ANALYZE \"nation$files\"")) + assertThat(query("ANALYZE \"nation$files\"")) // The error message isn't clear to the user, but it doesn't matter - .hasMessage("Cannot record write for catalog not part of transaction"); - assertThatThrownBy(() -> query("ANALYZE \"nation$snapshots\"")) + .nonTrinoExceptionFailure().hasMessage("Cannot record write for catalog not part of transaction"); + assertThat(query("ANALYZE \"nation$snapshots\"")) // The error message isn't clear to the user, but it doesn't matter - .hasMessage("Cannot record write for catalog not part of transaction"); + .nonTrinoExceptionFailure().hasMessage("Cannot record write for catalog not part of transaction"); } @Test @@ -694,8 +693,8 @@ public void testDropStatsSnapshot() assertUpdate("CREATE TABLE " + tableName + " (a) AS VALUES 11", 1); long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); - assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE DROP_EXTENDED_STATS".formatted(tableName, snapshotId))) - .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); + assertThat(query("ALTER TABLE \"%s@%d\" EXECUTE DROP_EXTENDED_STATS".formatted(tableName, snapshotId))) + .failure().hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -705,10 +704,10 @@ public void testDropStatsSnapshot() @Test public void testDropStatsSystemTable() { - assertThatThrownBy(() -> query("ALTER TABLE \"nation$files\" EXECUTE DROP_EXTENDED_STATS")) - .hasMessage("This connector does not support table procedures"); - assertThatThrownBy(() -> query("ALTER TABLE \"nation$snapshots\" EXECUTE DROP_EXTENDED_STATS")) - .hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$files\" EXECUTE DROP_EXTENDED_STATS")) + .failure().hasMessage("This connector does not support table procedures"); + assertThat(query("ALTER TABLE \"nation$snapshots\" EXECUTE DROP_EXTENDED_STATS")) + .failure().hasMessage("This connector does not support table procedures"); } @Test 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 b2b54ef823590..47b892a06d84b 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 @@ -98,7 +98,6 @@ import static org.apache.iceberg.FileFormat.ORC; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -409,7 +408,7 @@ public void testMultipleEqualityDeletesWithNestedFields() } // TODO: support read equality deletes with nested fields(https://github.com/trinodb/trino/issues/18625) - assertThatThrownBy(() -> query("SELECT * FROM " + tableName)).hasMessageContaining("Multiple entries with same key"); + assertThat(query("SELECT * FROM " + tableName)).failure().hasMessageContaining("Multiple entries with same key"); assertUpdate("DROP TABLE " + tableName); } @@ -577,7 +576,8 @@ public void testDowngradingV2TableToV1Fails() String tableName = "test_downgrading_v2_table_to_v1_fails_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " WITH (format_version = 2) AS SELECT * FROM tpch.tiny.nation", 25); assertThat(loadTable(tableName).operations().current().formatVersion()).isEqualTo(2); - assertThatThrownBy(() -> query("ALTER TABLE " + tableName + " SET PROPERTIES format_version = 1")) + assertThat(query("ALTER TABLE " + tableName + " SET PROPERTIES format_version = 1")) + .failure() .hasMessage("Failed to set new property values") .rootCause() .hasMessage("Cannot downgrade v2 table to v1"); @@ -589,8 +589,8 @@ public void testUpgradingToInvalidVersionFails() String tableName = "test_upgrading_to_invalid_version_fails_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " WITH (format_version = 2) AS SELECT * FROM tpch.tiny.nation", 25); assertThat(loadTable(tableName).operations().current().formatVersion()).isEqualTo(2); - assertThatThrownBy(() -> query("ALTER TABLE " + tableName + " SET PROPERTIES format_version = 42")) - .hasMessage("Unable to set catalog 'iceberg' table property 'format_version' to [42]: format_version must be between 1 and 2"); + assertThat(query("ALTER TABLE " + tableName + " SET PROPERTIES format_version = 42")) + .failure().hasMessage("Unable to set catalog 'iceberg' table property 'format_version' to [42]: format_version must be between 1 and 2"); } @Test diff --git a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxQueries.java b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxQueries.java index 9c852b871f40e..14ab12022db0a 100644 --- a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxQueries.java +++ b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxQueries.java @@ -65,7 +65,7 @@ public void teardown() public void testShowSchemas() { assertThat(assertions.query("SHOW SCHEMAS")) - .matches(result -> result.getOnlyColumnAsSet().equals(ImmutableSet.of(INFORMATION_SCHEMA, JMX_SCHEMA_NAME, HISTORY_SCHEMA_NAME))); + .result().onlyColumnAsSet().isEqualTo(Set.of(INFORMATION_SCHEMA, JMX_SCHEMA_NAME, HISTORY_SCHEMA_NAME)); } @Test @@ -76,7 +76,7 @@ public void testShowTables() .collect(toImmutableSet()); assertThat(assertions.query("SHOW TABLES")) - .matches(result -> result.getOnlyColumnAsSet().containsAll(standardNamesLower)); + .result().onlyColumnAsSet().containsAll(standardNamesLower); } @Test @@ -112,9 +112,9 @@ public void testQueryCumulativeTable() .succeeds(); assertThat(assertions.query("SELECT * FROM \"java.lang:*\"")) - .matches(result -> result.getRowCount() > 1); + .result().rowCount().isGreaterThan(1); assertThat(assertions.query("SELECT * FROM \"jAVA.LANg:*\"")) - .matches(result -> result.getRowCount() > 1); + .result().rowCount().isGreaterThan(1); } } diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConnectorTest.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConnectorTest.java index f4451fc375c89..d42d3c0a8549c 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConnectorTest.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConnectorTest.java @@ -65,7 +65,6 @@ import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assumptions.abort; public class TestKafkaConnectorTest @@ -428,8 +427,8 @@ public void testInsertNegativeDate() public void testInsertArray() { // Override because the base test uses CREATE TABLE statement that is unsupported in Kafka connector - assertThatThrownBy(() -> query("INSERT INTO " + TABLE_INSERT_ARRAY + " (a) VALUES (ARRAY[null])")) - .hasMessage("Unsupported column type 'array(double)' for column 'a'"); + assertThat(query("INSERT INTO " + TABLE_INSERT_ARRAY + " (a) VALUES (ARRAY[null])")) + .nonTrinoExceptionFailure().hasMessage("Unsupported column type 'array(double)' for column 'a'"); abort("not supported"); } 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 48363c5d2f452..39562d9becf66 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 @@ -430,11 +430,11 @@ public void testCreateTableWithLongTableName() assertUpdate("DROP TABLE " + validTableName); String invalidTableName = baseTableName + "z".repeat(256 - baseTableName.length() + 1); - assertThatThrownBy(() -> query("CREATE TABLE " + invalidTableName + "(" + + assertThat(query("CREATE TABLE " + invalidTableName + "(" + "id INT WITH (primary_key=true)," + "a VARCHAR)" + "WITH (partition_by_hash_columns = ARRAY['id'], partition_by_hash_buckets = 2)")) - .hasMessageContaining("invalid table name"); + .failure().hasMessageContaining("invalid table name"); assertThat(getQueryRunner().tableExists(getSession(), validTableName)).isFalse(); } 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 3872e0a6923e7..15f4a0299333e 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 @@ -295,8 +295,8 @@ public void testNativeQueryCreateStatement() { // Override because MariaDB returns a ResultSet metadata with no columns for CREATE statement. assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE tpch.numbers(n INTEGER)'))")) - .hasMessageContaining("descriptor has no fields"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE tpch.numbers(n INTEGER)'))")) + .nonTrinoExceptionFailure().hasMessageContaining("descriptor has no fields"); assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); } @@ -309,8 +309,8 @@ public void testNativeQueryInsertStatementTableExists() // The query fails because there are no columns, but even if columns were not required, the query would fail // to execute in MariaDB because the connector wraps it in additional syntax, which causes syntax error. try (TestTable testTable = simpleTable()) { - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) - .hasMessageContaining("descriptor has no fields"); + assertThat(query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) + .nonTrinoExceptionFailure().hasMessageContaining("descriptor has no fields"); assertQuery("SELECT * FROM " + testTable.getName(), "VALUES 1, 2"); } } 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 fdde9a47465c1..e3d3250b5660b 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 @@ -692,8 +692,8 @@ public void testDbRefMissingField() client.getDatabase("test").getCollection(tableName).insertOne(document); // TODO Fix MongoPageSource to throw TrinoException - assertThatThrownBy(() -> query("SELECT * FROM test." + tableName)) - .hasMessageContaining("DBRef should have 3 fields : row(databaseName varchar, collectionName varchar)"); + assertThat(query("SELECT * FROM test." + tableName)) + .nonTrinoExceptionFailure().hasMessageContaining("DBRef should have 3 fields : row(databaseName varchar, collectionName varchar)"); assertUpdate("DROP TABLE test." + tableName); } 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 11728f6af59c9..7cd50783cd2c7 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 @@ -517,8 +517,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() { // override because MySQL succeeds in preparing query, and then fails because of no metadata available assertThat(getQueryRunner().tableExists(getSession(), "non_existent_table")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) - .hasMessageContaining("Query not supported: ResultSetMetaData not available for query: INSERT INTO non_existent_table VALUES (1)"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) + .nonTrinoExceptionFailure().hasMessageContaining("Query not supported: ResultSetMetaData not available for query: INSERT INTO non_existent_table VALUES (1)"); } @Test @@ -526,8 +526,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() public void testNativeQueryIncorrectSyntax() { // override because MySQL succeeds in preparing query, and then fails because of no metadata available - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) - .hasMessageContaining("Query not supported: ResultSetMetaData not available for query: some wrong syntax"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + .nonTrinoExceptionFailure().hasMessageContaining("Query not supported: ResultSetMetaData not available for query: some wrong syntax"); } @Test diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java index 3e727a8facb06..9bc69c8075b8c 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java @@ -1877,12 +1877,12 @@ public void testQueryTableFunction() "VALUES '[]'"); // syntax error - assertThatThrownBy(() -> query("SELECT * " + + assertThat(query("SELECT * " + format("FROM TABLE(%s.system.raw_query(", catalogName) + "schema => 'tpch', " + "index => 'nation', " + "query => 'wrong syntax')) t(result)")) - .hasMessageContaining("json_parse_exception"); + .failure().hasMessageContaining("json_parse_exception"); } protected void assertTableDoesNotExist(String name) 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 1c0adf2c3604f..03f4968e6ba6a 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 @@ -403,8 +403,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() { // override because Oracle succeeds in preparing query, and then fails because of no metadata available assertThat(getQueryRunner().tableExists(getSession(), "non_existent_table")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) - .hasMessageContaining("Query not supported: ResultSetMetaData not available for query: INSERT INTO non_existent_table VALUES (1)"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) + .nonTrinoExceptionFailure().hasMessageContaining("Query not supported: ResultSetMetaData not available for query: INSERT INTO non_existent_table VALUES (1)"); } @Test @@ -412,8 +412,8 @@ public void testNativeQueryInsertStatementTableDoesNotExist() public void testNativeQueryIncorrectSyntax() { // override because Oracle succeeds in preparing query, and then fails because of no metadata available - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) - .hasMessageContaining("Query not supported: ResultSetMetaData not available for query: some wrong syntax"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) + .nonTrinoExceptionFailure().hasMessageContaining("Query not supported: ResultSetMetaData not available for query: some wrong syntax"); } @Override 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 50a3d2335e148..71e5ec9e3e1b5 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 @@ -150,15 +150,15 @@ public void testArithmeticPredicatePushdown() // multiplication/division/modulo by zero assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey * 0 != 0")) .isFullyPushedDown(); - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey / 0 = 0")) - .satisfies(this::verifyDivisionByZeroFailure); - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey % 0 = 0")) - .satisfies(this::verifyDivisionByZeroFailure); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey / 0 = 0")) + .failure().satisfies(this::verifyDivisionByZeroFailure); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey % 0 = 0")) + .failure().satisfies(this::verifyDivisionByZeroFailure); // Expression that evaluates to 0 for some rows on RHS of modulus - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) / (regionkey - 1) = 2")) - .satisfies(this::verifyDivisionByZeroFailure); - assertThatThrownBy(() -> query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % (regionkey - 1) = 2")) - .satisfies(this::verifyDivisionByZeroFailure); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) / (regionkey - 1) = 2")) + .failure().satisfies(this::verifyDivisionByZeroFailure); + assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey > 0 AND (nationkey - regionkey) % (regionkey - 1) = 2")) + .failure().satisfies(this::verifyDivisionByZeroFailure); // multiplicative/divisive identity assertThat(query("SELECT nationkey, name, regionkey FROM nation WHERE nationkey * 1 = nationkey")) diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotConnectorSmokeTest.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotConnectorSmokeTest.java index d9ba2b796a157..58bf2455cd3b7 100644 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotConnectorSmokeTest.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotConnectorSmokeTest.java @@ -2036,13 +2036,15 @@ public void testAggregationPushdown() // Ensure that count() is not pushed down even if the query contains a matching grouping column assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> query("SELECT COUNT(long_col) FROM \"SELECT long_col FROM " + ALL_TYPES_TABLE + " GROUP BY long_col\"")) + // TODO verify the failure is TrinoException (eg. asserThat(query(....)).failure()...) + .isThrownBy(() -> computeActual("SELECT COUNT(long_col) FROM \"SELECT long_col FROM " + ALL_TYPES_TABLE + " GROUP BY long_col\"")) .withRootCauseInstanceOf(RuntimeException.class) .withMessage("Operation not supported for DISTINCT aggregation function"); // Ensure that count() with grouping columns is not pushed down even if the query contains a matching grouping column assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> query("SELECT bool_col, COUNT(long_col) FROM \"SELECT bool_col, long_col FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col, long_col\"")) + // TODO verify the failure is TrinoException (eg. asserThat(query(....)).failure()...) + .isThrownBy(() -> computeActual("SELECT bool_col, COUNT(long_col) FROM \"SELECT bool_col, long_col FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col, long_col\"")) .withRootCauseInstanceOf(RuntimeException.class) .withMessage("Operation not supported for DISTINCT aggregation function"); diff --git a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java index 52a96949499cc..0c4b5a002a39d 100644 --- a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java +++ b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java @@ -658,8 +658,8 @@ public void testTimestampWithTimeZoneOverflow() // The max timestamp with time zone value in Redshift is larger than Trino try (TestTable table = new TestTable(getRedshiftExecutor(), TEST_SCHEMA + ".timestamp_tz_max", "(ts timestamptz)", ImmutableList.of("TIMESTAMP '294276-12-31 23:59:59' AT TIME ZONE 'UTC'"))) { - assertThatThrownBy(() -> query("SELECT * FROM " + table.getName())) - .hasMessage("Millis overflow: 9224318015999000"); + assertThat(query("SELECT * FROM " + table.getName())) + .failure().hasMessage("Millis overflow: 9224318015999000"); } } 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 5baf089993f88..45799d56354e3 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 @@ -352,8 +352,8 @@ public void testNativeQueryCreateStatement() // The query fails because there are no columns, but even if columns were not required, the query would fail // to execute in SingleStore because the connector wraps it in additional syntax, which causes syntax error. assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) - .hasMessageContaining("descriptor has no fields"); + assertThat(query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) + .nonTrinoExceptionFailure().hasMessageContaining("descriptor has no fields"); assertThat(getQueryRunner().tableExists(getSession(), "numbers")).isFalse(); } @@ -366,8 +366,8 @@ public void testNativeQueryInsertStatementTableExists() // The query fails because there are no columns, but even if columns were not required, the query would fail // to execute in SingleStore because the connector wraps it in additional syntax, which causes syntax error. try (TestTable testTable = simpleTable()) { - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) - .hasMessageContaining("descriptor has no fields"); + assertThat(query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) + .nonTrinoExceptionFailure().hasMessageContaining("descriptor has no fields"); assertQuery("SELECT * FROM " + testTable.getName(), "VALUES 1, 2"); } } diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java index 8dbf8c84a493f..77f8d6798c4f3 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java @@ -306,8 +306,8 @@ public void testCreateTableAsTable() .matches("SELECT * FROM tpch.tiny.nation"); // Verify that hidden column is not present in the created table - assertThatThrownBy(() -> query("SELECT min(row_number) FROM n")) - .hasMessage("line 1:12: Column 'row_number' cannot be resolved"); + assertThat(query("SELECT min(row_number) FROM n")) + .failure().hasMessage("line 1:12: Column 'row_number' cannot be resolved"); assertUpdate(getSession(), "DROP TABLE n"); } @@ -328,8 +328,8 @@ public void testInsertTableIntoTable() .matches("SELECT * FROM tpch.tiny.nation LIMIT 0"); // Verify that the hidden column is not present in the created table - assertThatThrownBy(() -> query("SELECT row_number FROM n")) - .hasMessage("line 1:8: Column 'row_number' cannot be resolved"); + assertThat(query("SELECT row_number FROM n")) + .failure().hasMessage("line 1:8: Column 'row_number' cannot be resolved"); // Insert values from the original table into the created table assertUpdate(getSession(), "INSERT INTO n TABLE tpch.tiny.nation", 25); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java index da3053dd8de5f..e827c93537d59 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java @@ -71,7 +71,6 @@ import static java.util.stream.Collectors.toList; import static java.util.stream.IntStream.range; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public abstract class AbstractTestEngineOnlyQueries extends AbstractTestQueryFramework @@ -6691,21 +6690,21 @@ SELECT abs('abc') .matches("VALUES 'cba'"); // inline functions must be declared before they are used - assertThatThrownBy(() -> query(""" + assertThat(query(""" WITH FUNCTION a(x integer) RETURNS integer RETURN b(x), FUNCTION b(x integer) RETURNS integer RETURN x * 2 SELECT a(10) """)) - .hasMessage("line 3:8: Function 'b' not registered"); + .failure().hasMessage("line 3:8: Function 'b' not registered"); // inline function cannot be recursive // note: mutual recursion is not supported either, but it is not tested due to the forward declaration limitation above - assertThatThrownBy(() -> query(""" + assertThat(query(""" WITH FUNCTION a(x integer) RETURNS integer RETURN a(x) SELECT a(10) """)) - .hasMessage("line 3:8: Recursive language functions are not supported: a(integer):integer"); + .failure().hasMessage("line 3:8: Recursive language functions are not supported: a(integer):integer"); } // ensure that JSON_TABLE runs properly in distributed mode (i.e., serialization of handles works correctly, etc) 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 d57970878e9eb..5912ba2df87ed 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 @@ -42,9 +42,12 @@ import io.trino.sql.planner.plan.OutputNode; import io.trino.sql.planner.plan.ProjectNode; import io.trino.sql.planner.plan.TableScanNode; +import io.trino.sql.query.QueryAssertions.QueryAssert; +import io.trino.testing.assertions.TrinoExceptionAssert; import io.trino.testing.sql.TestTable; import io.trino.testing.sql.TestView; import io.trino.tpch.TpchTable; +import org.assertj.core.api.AssertProvider; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.RepeatedTest; @@ -613,30 +616,30 @@ public void testVarcharCastToDateInPredicate() "varchar_as_date_pred", "(a varchar)", List.of("'2005-06-bad-date'", "'2005-09-10'"))) { - assertThatThrownBy(() -> query("SELECT a FROM %s WHERE CAST(a AS date) < DATE '2005-09-10'".formatted(table.getName()))) - .hasMessage("Value cannot be cast to date: 2005-06-bad-date"); + assertThat(query("SELECT a FROM %s WHERE CAST(a AS date) < DATE '2005-09-10'".formatted(table.getName()))) + .failure().hasMessage("Value cannot be cast to date: 2005-06-bad-date"); verifyResultOrFailure( - () -> query("SELECT a FROM %s WHERE CAST(a AS date) = DATE '2005-09-10'".formatted(table.getName())), - queryAssert -> assertThat(queryAssert) + query("SELECT a FROM %s WHERE CAST(a AS date) = DATE '2005-09-10'".formatted(table.getName())), + queryAssert -> queryAssert .skippingTypesCheck() .matches("VALUES '2005-09-10'"), - failure -> assertThat(failure) + failureAssert -> failureAssert .hasMessage("Value cannot be cast to date: 2005-06-bad-date")); // This failure isn't guaranteed: a row may be filtered out on the connector side with a derived predicate on a varchar column. verifyResultOrFailure( - () -> query("SELECT a FROM %s WHERE CAST(a AS date) != DATE '2005-9-1'".formatted(table.getName())), - queryAssert -> assertThat(queryAssert) + query("SELECT a FROM %s WHERE CAST(a AS date) != DATE '2005-9-1'".formatted(table.getName())), + queryAssert -> queryAssert .skippingTypesCheck() .matches("VALUES '2005-09-10'"), - failure -> assertThat(failure) + failureAssert -> failureAssert .hasMessage("Value cannot be cast to date: 2005-06-bad-date")); // This failure isn't guaranteed: a row may be filtered out on the connector side with a derived predicate on a varchar column. verifyResultOrFailure( - () -> query("SELECT a FROM %s WHERE CAST(a AS date) > DATE '2022-08-10'".formatted(table.getName())), - queryAssert -> assertThat(queryAssert) + query("SELECT a FROM %s WHERE CAST(a AS date) > DATE '2022-08-10'".formatted(table.getName())), + queryAssert -> queryAssert .skippingTypesCheck() .returnsEmptyResult(), - failure -> assertThat(failure) + failureAssert -> failureAssert .hasMessage("Value cannot be cast to date: 2005-06-bad-date")); } try (TestTable table = new TestTable( @@ -651,21 +654,20 @@ public void testVarcharCastToDateInPredicate() } } - private static void verifyResultOrFailure(Supplier callback, Consumer verifyResults, Consumer verifyFailure) + private static void verifyResultOrFailure(AssertProvider queryAssertProvider, Consumer verifyResults, Consumer verifyFailure) { - requireNonNull(callback, "callback is null"); requireNonNull(verifyResults, "verifyResults is null"); requireNonNull(verifyFailure, "verifyFailure is null"); - T result; + QueryAssert queryAssert = assertThat(queryAssertProvider); try { - result = callback.get(); + var ignored = queryAssert.result(); } catch (Throwable t) { - verifyFailure.accept(t); + verifyFailure.accept(queryAssert.failure()); return; } - verifyResults.accept(result); + verifyResults.accept(queryAssert); } @Test @@ -697,10 +699,10 @@ public void testSelectVersionOfNonExistentTable() String catalog = getSession().getCatalog().orElseThrow(); String schema = getSession().getSchema().orElseThrow(); String tableName = "foo_" + randomNameSuffix(); - assertThatThrownBy(() -> query("SELECT * FROM " + tableName + " FOR TIMESTAMP AS OF TIMESTAMP '2021-03-01 00:00:01'")) - .hasMessage(format("line 1:15: Table '%s.%s.%s' does not exist", catalog, schema, tableName)); - assertThatThrownBy(() -> query("SELECT * FROM " + tableName + " FOR VERSION AS OF 'version1'")) - .hasMessage(format("line 1:15: Table '%s.%s.%s' does not exist", catalog, schema, tableName)); + assertThat(query("SELECT * FROM " + tableName + " FOR TIMESTAMP AS OF TIMESTAMP '2021-03-01 00:00:01'")) + .failure().hasMessage(format("line 1:15: Table '%s.%s.%s' does not exist", catalog, schema, tableName)); + assertThat(query("SELECT * FROM " + tableName + " FOR VERSION AS OF 'version1'")) + .failure().hasMessage(format("line 1:15: Table '%s.%s.%s' does not exist", catalog, schema, tableName)); } /** @@ -4216,9 +4218,9 @@ public void testInsertArray() String tableName = "test_insert_array_" + randomNameSuffix(); if (!hasBehavior(SUPPORTS_ARRAY)) { - assertThatThrownBy(() -> query("CREATE TABLE " + tableName + " (a array(bigint))")) + assertThat(query("CREATE TABLE " + tableName + " (a array(bigint))")) // TODO Unify failure message across connectors - .hasMessageMatching("[Uu]nsupported (column )?type: \\Qarray(bigint)"); + .failure().hasMessageMatching("[Uu]nsupported (column )?type: \\Qarray(bigint)"); abort("not supported"); } @@ -6424,6 +6426,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name + "(x double) RETURNS double COMMENT 't88' RETURN x * 8.8"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") @@ -6442,6 +6445,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name2 + "(s varchar) RETURNS varchar RETURN 'Hello ' || s"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name, "bigint", "integer", "scalar", true, "t42") @@ -6460,6 +6464,7 @@ public void testCreateFunction() assertUpdate("CREATE FUNCTION " + name3 + "() RETURNS double NOT DETERMINISTIC RETURN random()"); assertThat(query("SHOW FUNCTIONS")) + .result() .skippingTypesCheck() .containsAll(resultBuilder(getSession()) .row(name3, "double", "", "scalar", false, "") diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseTestParquetWithBloomFilters.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseTestParquetWithBloomFilters.java index 4cd93f25296fb..450665f1eef01 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseTestParquetWithBloomFilters.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseTestParquetWithBloomFilters.java @@ -38,7 +38,7 @@ public abstract class BaseTestParquetWithBloomFilters public void verifyBloomFilterEnabled() { assertThat(query(format("SHOW SESSION LIKE '%s.parquet_use_bloom_filter'", getSession().getCatalog().orElseThrow()))) - .skippingTypesCheck() + .result() .matches(result -> result.getRowCount() == 1) .matches(result -> { String value = (String) result.getMaterializedRows().get(0).getField(1); diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestErrorThrowableInQuery.java b/testing/trino-tests/src/test/java/io/trino/execution/TestErrorThrowableInQuery.java index 7d5db6e258da9..f4b03974f2956 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestErrorThrowableInQuery.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestErrorThrowableInQuery.java @@ -34,7 +34,6 @@ import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Test scenarios where query fails unexpected {@link Error} during execution. This is worth testing @@ -102,19 +101,19 @@ public void testSystemRuntimeQueriesWorksAfterStackOverflowErrorDuringAnalyzeInE // This tests simulates case when StackOverflowError is throwing during planning because of size and shape of // the query. It is possible to construct query such that it makes its way through parsed and then fail during // planning but it is hard to do that in a predictable way, hence we use mock connector here. - assertThatThrownBy(() -> query("EXPLAIN SELECT test_varchar FROM stack_overflow_during_planning")) - .hasMessageContaining("statement is too large (stack overflow during analysis)"); + assertThat(query("EXPLAIN SELECT test_varchar FROM stack_overflow_during_planning")) + .failure().hasMessageContaining("statement is too large (stack overflow during analysis)"); - assertThat(query("SELECT * FROM system.runtime.queries")).matches(result -> result.getRowCount() > 0); + assertThat(query("SELECT * FROM system.runtime.queries")).result().rowCount().isGreaterThan(0); } @Test public void testSystemRuntimeQueriesWorksAfterClassFormatErrorDuringAnalyzeInExplain() throws Exception { - assertThatThrownBy(() -> query("EXPLAIN SELECT test_varchar FROM class_format_error_during_planning")) - .hasMessageContaining("java.lang.ClassFormatError: Bad class format!!!!!!!!!!"); + assertThat(query("EXPLAIN SELECT test_varchar FROM class_format_error_during_planning")) + .nonTrinoExceptionFailure().hasMessageContaining("java.lang.ClassFormatError: Bad class format!!!!!!!!!!"); - assertThat(query("SELECT * FROM system.runtime.queries")).matches(result -> result.getRowCount() > 0); + assertThat(query("SELECT * FROM system.runtime.queries")).result().rowCount().isGreaterThan(0); } } diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestTableRedirection.java b/testing/trino-tests/src/test/java/io/trino/execution/TestTableRedirection.java index 61b1e4da30651..5bb8496ac4299 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestTableRedirection.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestTableRedirection.java @@ -54,7 +54,6 @@ import static java.lang.String.format; import static java.util.Collections.emptyIterator; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestTableRedirection extends AbstractTestQueryFramework @@ -206,7 +205,8 @@ public void testTableScans() "SELECT 1 WHERE 1=0", verifySingleTableScan(SCHEMA_TWO, VALID_REDIRECTION_TARGET)); - assertThatThrownBy(() -> query(format("SELECT c0 FROM %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + assertThat(query(format("SELECT c0 FROM %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + .failure() .hasMessageContaining( "Table '%s' redirected to '%s', but the target table '%s' does not exist", new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, BAD_REDIRECTION_SRC), @@ -218,14 +218,16 @@ public void testTableScans() "SELECT 1 WHERE 1=0", verifySingleTableScan(SCHEMA_ONE, TABLE_FOO)); - assertThatThrownBy(() -> query(format("SELECT c0 FROM %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + assertThat(query(format("SELECT c0 FROM %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + .failure() .hasMessageContaining( "Table redirections form a loop: %s -> %s -> %s", new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, REDIRECTION_LOOP_PING), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, REDIRECTION_LOOP_PONG), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, REDIRECTION_LOOP_PING)); - assertThatThrownBy(() -> query(format("SELECT c4 FROM %s.%s", SCHEMA_THREE, REDIRECTION_CHAIN.get(0)))) + assertThat(query(format("SELECT c4 FROM %s.%s", SCHEMA_THREE, REDIRECTION_CHAIN.get(0)))) + .failure() .hasMessageContaining( "Table redirected too many times (10): [%s]", REDIRECTION_CHAIN.stream() @@ -342,14 +344,16 @@ public void testShowCreate() String showCreateValidTarget = (String) computeScalar(format("SHOW CREATE TABLE %s.%s", SCHEMA_TWO, VALID_REDIRECTION_TARGET)); assertThat(showCreateValidTarget).isEqualTo(showCreateValidSource.replace(SCHEMA_ONE + "." + VALID_REDIRECTION_SRC, SCHEMA_TWO + "." + VALID_REDIRECTION_TARGET)); - assertThatThrownBy(() -> query(format("SHOW CREATE TABLE %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + assertThat(query(format("SHOW CREATE TABLE %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + .failure() .hasMessageContaining( "Table '%s' redirected to '%s', but the target table '%s' does not exist", new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, BAD_REDIRECTION_SRC), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE)); - assertThatThrownBy(() -> query(format("SHOW CREATE TABLE %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + assertThat(query(format("SHOW CREATE TABLE %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + .failure() .hasMessageContaining("Table redirections form a loop"); } @@ -359,14 +363,16 @@ public void testDescribeTable() assertThat(query(format("DESCRIBE %s.%s", SCHEMA_ONE, VALID_REDIRECTION_SRC))) .matches(format("DESCRIBE %s.%s", SCHEMA_TWO, VALID_REDIRECTION_TARGET)); - assertThatThrownBy(() -> query(format("DESCRIBE %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + assertThat(query(format("DESCRIBE %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + .failure() .hasMessageContaining( "Table '%s' redirected to '%s', but the target table '%s' does not exist", new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, BAD_REDIRECTION_SRC), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE)); - assertThatThrownBy(() -> query(format("DESCRIBE %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + assertThat(query(format("DESCRIBE %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + .failure() .hasMessageContaining("Table redirections form a loop"); } @@ -379,15 +385,16 @@ public void testShowColumns() + row(C2, BIGINT.getDisplayName(), "", "") + "," + row(C3, BIGINT.getDisplayName(), "", "")); - assertThatThrownBy(() -> query(format("SHOW COLUMNS FROM %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + assertThat(query(format("SHOW COLUMNS FROM %s.%s", SCHEMA_ONE, BAD_REDIRECTION_SRC))) + .failure() .hasMessageContaining( "Table '%s' redirected to '%s', but the target table '%s' does not exist", new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, BAD_REDIRECTION_SRC), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE), new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE)); - assertThatThrownBy(() -> query(format("SHOW COLUMNS FROM %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) - .hasMessageContaining("Table redirections form a loop"); + assertThat(query(format("SHOW COLUMNS FROM %s.%s", SCHEMA_ONE, REDIRECTION_LOOP_PING))) + .failure().hasMessageContaining("Table redirections form a loop"); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/tests/BaseQueryAssertionsTest.java b/testing/trino-tests/src/test/java/io/trino/tests/BaseQueryAssertionsTest.java index d9bd147f9cd82..7d3de52531c18 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/BaseQueryAssertionsTest.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/BaseQueryAssertionsTest.java @@ -119,11 +119,11 @@ public void testReturnsEmptyResult() QueryAssert queryAssert = assertThat(query("VALUES 'foobar'")); assertThatThrownBy(queryAssert::returnsEmptyResult) - .hasMessageContaining("[Rows for query [VALUES 'foobar']] \nExpecting empty but was: [[foobar]]"); + .hasMessageContaining("[Rows for query [VALUES 'foobar']] \nExpecting empty but was: (foobar)"); queryAssert = assertThat(query("VALUES 'foo', 'bar'")); assertThatThrownBy(queryAssert::returnsEmptyResult) - .hasMessageContaining("[Rows for query [VALUES 'foo', 'bar']] \nExpecting empty but was: [[foo], [bar]]"); + .hasMessageContaining("[Rows for query [VALUES 'foo', 'bar']] \nExpecting empty but was: (foo), (bar)"); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestExcludeColumnsFunction.java b/testing/trino-tests/src/test/java/io/trino/tests/TestExcludeColumnsFunction.java index 774e0d3df0227..f492813b5f17b 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestExcludeColumnsFunction.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestExcludeColumnsFunction.java @@ -21,7 +21,6 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestExcludeColumnsFunction extends AbstractTestQueryFramework @@ -61,45 +60,45 @@ FROM TABLE(exclude_columns( @Test public void testInvalidArgument() { - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => CAST(null AS DESCRIPTOR))) """)) - .hasMessage("COLUMNS descriptor is null"); + .failure().hasMessage("COLUMNS descriptor is null"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR())) """)) - .hasMessage("line 4:21: Invalid descriptor argument COLUMNS. Descriptors should be formatted as 'DESCRIPTOR(name [type], ...)'"); + .failure().hasMessage("line 4:21: Invalid descriptor argument COLUMNS. Descriptors should be formatted as 'DESCRIPTOR(name [type], ...)'"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR(foo, comment, bar))) """)) - .hasMessage("Excluded columns: [foo, bar] not present in the table"); + .failure().hasMessage("Excluded columns: [foo, bar] not present in the table"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR(nationkey bigint, comment))) """)) - .hasMessage("COLUMNS descriptor contains types"); + .failure().hasMessage("COLUMNS descriptor contains types"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR(nationkey, name, regionkey, comment))) """)) - .hasMessage("All columns are excluded"); + .failure().hasMessage("All columns are excluded"); } @Test @@ -134,21 +133,21 @@ public void testHiddenColumn() assertThat(query("SELECT row_number FROM tpch.tiny.region")).matches("SELECT * FROM UNNEST(sequence(0, 4))"); // the hidden column is not provided to the function - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT row_number FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR(comment))) """)) - .hasMessage("line 1:8: Column 'row_number' cannot be resolved"); + .failure().hasMessage("line 1:8: Column 'row_number' cannot be resolved"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(exclude_columns( input => TABLE(tpch.tiny.nation), columns => DESCRIPTOR(row_number))) """)) - .hasMessage("Excluded columns: [row_number] not present in the table"); + .failure().hasMessage("Excluded columns: [row_number] not present in the table"); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java b/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java index b1688f3e3a23c..2b3af5f58c1d0 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java @@ -22,7 +22,6 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestSequenceFunction extends AbstractTestQueryFramework @@ -81,50 +80,50 @@ FROM TABLE(sequence(stop => 10)) @Test public void testInvalidArgument() { - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(sequence( start => -5, stop => 10, step => -2)) """)) - .hasMessage("Step must be positive for sequence [-5, 10]"); + .failure().hasMessage("Step must be positive for sequence [-5, 10]"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(sequence( start => 10, stop => -5, step => 2)) """)) - .hasMessage("Step must be negative for sequence [10, -5]"); + .failure().hasMessage("Step must be negative for sequence [10, -5]"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(sequence( start => null, stop => -5, step => 2)) """)) - .hasMessage("Start is null"); + .failure().hasMessage("Start is null"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(sequence( start => 10, stop => null, step => 2)) """)) - .hasMessage("Stop is null"); + .failure().hasMessage("Stop is null"); - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(sequence( start => 10, stop => -5, step => null)) """)) - .hasMessage("Step is null"); + .failure().hasMessage("Step is null"); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestTableFunctionInvocation.java b/testing/trino-tests/src/test/java/io/trino/tests/TestTableFunctionInvocation.java index 7fc083b471281..ad37e70c6ec78 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestTableFunctionInvocation.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestTableFunctionInvocation.java @@ -39,7 +39,6 @@ import static io.trino.testing.TestingAccessControlManager.privilege; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestTableFunctionInvocation extends AbstractTestQueryFramework @@ -699,11 +698,11 @@ FROM TABLE(system.constant(5 * 4, 3)) .matches("VALUES (20), (20), (20)"); // value out of range for INTEGER type: Integer.MAX_VALUE + 1 - assertThatThrownBy(() -> query(""" + assertThat(query(""" SELECT * FROM TABLE(system.constant(2147483648, 3)) """)) - .hasMessage("line 2:28: Cannot cast type bigint to integer"); + .failure().hasMessage("line 2:28: Cannot cast type bigint to integer"); assertThat(query(""" SELECT count(*), count(DISTINCT constant_column), min(constant_column) diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestTablesample.java b/testing/trino-tests/src/test/java/io/trino/tests/TestTablesample.java index a46b6bddcde03..bc7e31508b071 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestTablesample.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestTablesample.java @@ -29,7 +29,6 @@ import static io.trino.plugin.tpch.TpchConnectorFactory.TPCH_SPLITS_PER_NODE; import static io.trino.spi.StandardErrorCode.INVALID_ARGUMENTS; import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -73,41 +72,45 @@ public void testTablesample() // 1% assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (1)")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(50L, 450L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(50L, 450L)); // 0.1% assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (1e-1)")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); // 0.1% as decimal assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (0.1)")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); // fraction as long decimal assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (0.000000000000000000001)")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(0L, 5L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(0L, 5L)); } @Test public void testNullRatio() { // NULL - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (NULL)")) + assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (NULL)")) + .failure() .hasErrorCode(INVALID_ARGUMENTS) .hasMessage("line 1:62: Sample percentage cannot be NULL"); // NULL integer - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS integer))")) + assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS integer))")) + .failure() .hasErrorCode(INVALID_ARGUMENTS) .hasMessage("line 1:62: Sample percentage cannot be NULL"); // NULL double - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS double))")) + assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS double))")) + .failure() .hasErrorCode(INVALID_ARGUMENTS) .hasMessage("line 1:62: Sample percentage cannot be NULL"); // NULL varchar - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS varchar))")) + assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (CAST(NULL AS varchar))")) + .failure() .hasErrorCode(TYPE_MISMATCH) .hasMessage("line 1:62: Sample percentage should be a numeric expression"); } @@ -115,7 +118,8 @@ public void testNullRatio() @Test public void testInvalidRatioType() { - assertTrinoExceptionThrownBy(() -> assertions.query("SELECT count(*) FROM tpch.sf1.orders TABLESAMPLE BERNOULLI (DATE '1970-01-02')")) + assertThat(assertions.query("SELECT count(*) FROM tpch.sf1.orders TABLESAMPLE BERNOULLI (DATE '1970-01-02')")) + .failure() .hasErrorCode(TYPE_MISMATCH) .hasMessage("line 1:61: Sample percentage should be a numeric expression"); } @@ -133,18 +137,18 @@ public void testInSubquery() // 1% assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders WHERE orderkey IN (SELECT orderkey FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (1))")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(50L, 450L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(50L, 450L)); // 0.1% assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders WHERE orderkey IN (SELECT orderkey FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (1e-1))")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); // 0.1% as decimal assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders WHERE orderkey IN (SELECT orderkey FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (0.1))")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(3L, 45L)); // fraction as long decimal assertThat(assertions.query("SELECT count(*) FROM tpch.tiny.orders WHERE orderkey IN (SELECT orderkey FROM tpch.tiny.orders TABLESAMPLE BERNOULLI (0.000000000000000000001))")) - .satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(0L, 5L)); + .result().satisfies(result -> assertThat((Long) result.getOnlyValue()).isBetween(0L, 5L)); } }