Skip to content

Commit

Permalink
Remove ValidateLimitWithPresortedInput
Browse files Browse the repository at this point in the history
It's not powerful enough to validate properties of plans
that get modified by predicate pushdown after AddExchanges
runs, resulting in false positives such as #16768
  • Loading branch information
martint committed Apr 4, 2023
1 parent 54953dc commit 47de746
Show file tree
Hide file tree
Showing 4 changed files with 36 additions and 345 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ public PlanSanityChecker(boolean forceSingleNode)
new ValidateAggregationsWithDefaultValues(forceSingleNode),
new ValidateScaledWritersUsage(),
new ValidateStreamingAggregations(),
new ValidateLimitWithPresortedInput(),
new DynamicFiltersChecker(),
new TableScanValidator(),
new TableExecuteStructureValidator())
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -20,38 +20,34 @@
import io.trino.connector.MockConnectorColumnHandle;
import io.trino.connector.MockConnectorFactory;
import io.trino.connector.MockConnectorTableHandle;
import io.trino.execution.warnings.WarningCollector;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.connector.ColumnMetadata;
import io.trino.spi.connector.ConnectorTableProperties;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SortingProperty;
import io.trino.spi.predicate.TupleDomain;
import io.trino.sql.PlannerContext;
import io.trino.sql.planner.Plan;
import io.trino.spi.type.RowType;
import io.trino.sql.planner.assertions.BasePlanTest;
import io.trino.sql.planner.assertions.PlanAssert;
import io.trino.sql.planner.assertions.PlanMatchPattern;
import io.trino.sql.planner.sanity.ValidateLimitWithPresortedInput;
import io.trino.sql.tree.LongLiteral;
import io.trino.testing.LocalQueryRunner;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.Test;

import java.util.List;
import java.util.Optional;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector;
import static io.trino.spi.connector.SortOrder.ASC_NULLS_FIRST;
import static io.trino.spi.connector.SortOrder.ASC_NULLS_LAST;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED;
import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer;
import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree;
import static io.trino.sql.planner.assertions.PlanMatchPattern.exchange;
import static io.trino.sql.planner.assertions.PlanMatchPattern.expression;
import static io.trino.sql.planner.assertions.PlanMatchPattern.filter;
import static io.trino.sql.planner.assertions.PlanMatchPattern.limit;
import static io.trino.sql.planner.assertions.PlanMatchPattern.output;
import static io.trino.sql.planner.assertions.PlanMatchPattern.project;
import static io.trino.sql.planner.assertions.PlanMatchPattern.sort;
import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan;
import static io.trino.sql.planner.assertions.PlanMatchPattern.topN;
Expand Down Expand Up @@ -79,6 +75,8 @@ public class TestPartialTopNWithPresortedInput
private static final ColumnHandle columnHandleA = new MockConnectorColumnHandle(columnNameA, VARCHAR);
private static final String columnNameB = "col_b";

private static final SchemaTableName nestedField = new SchemaTableName(TEST_SCHEMA, "with_nested_field");

@Override
protected LocalQueryRunner createLocalQueryRunner()
{
Expand All @@ -98,6 +96,9 @@ protected LocalQueryRunner createLocalQueryRunner()
Optional.empty(),
ImmutableList.of(new SortingProperty<>(columnHandleA, ASC_NULLS_FIRST)));
}
else if (tableHandle.getTableName().equals(nestedField)) {
return new ConnectorTableProperties();
}
throw new IllegalArgumentException();
})
.withGetColumns(schemaTableName -> {
Expand All @@ -106,6 +107,10 @@ protected LocalQueryRunner createLocalQueryRunner()
new ColumnMetadata(columnNameA, VARCHAR),
new ColumnMetadata(columnNameB, VARCHAR));
}
else if (schemaTableName.equals(nestedField)) {
return ImmutableList.of(
new ColumnMetadata("nested", RowType.from(ImmutableList.of(RowType.field("k", INTEGER)))));
}
throw new IllegalArgumentException();
})
.build();
Expand All @@ -117,9 +122,7 @@ protected LocalQueryRunner createLocalQueryRunner()
public void testWithSortedTable()
{
List<PlanMatchPattern.Ordering> orderBy = ImmutableList.of(sort("t_col_a", ASCENDING, FIRST));
assertPlanWithValidation(
"SELECT col_a FROM table_a ORDER BY 1 ASC NULLS FIRST LIMIT 10",
output(
assertDistributedPlan("SELECT col_a FROM table_a ORDER BY 1 ASC NULLS FIRST LIMIT 10", output(
topN(10, orderBy, FINAL,
exchange(LOCAL, GATHER, ImmutableList.of(),
exchange(REMOTE, GATHER, ImmutableList.of(),
Expand All @@ -132,19 +135,15 @@ public void testWithSortedTable()
.collect(toImmutableList()),
tableScan("table_a", ImmutableMap.of("t_col_a", "col_a"))))))));

assertPlanWithValidation(
"SELECT col_a FROM table_a ORDER BY 1 ASC NULLS FIRST",
output(
assertDistributedPlan("SELECT col_a FROM table_a ORDER BY 1 ASC NULLS FIRST", output(
exchange(REMOTE, GATHER, orderBy,
exchange(LOCAL, GATHER, orderBy,
sort(orderBy,
exchange(REMOTE, REPARTITION,
tableScan("table_a", ImmutableMap.of("t_col_a", "col_a"))))))));

orderBy = ImmutableList.of(sort("t_col_a", ASCENDING, LAST));
assertPlanWithValidation(
"SELECT col_a FROM table_a ORDER BY 1 ASC NULLS LAST LIMIT 10",
output(
assertDistributedPlan("SELECT col_a FROM table_a ORDER BY 1 ASC NULLS LAST LIMIT 10", output(
topN(10, orderBy, FINAL,
exchange(LOCAL, GATHER, ImmutableList.of(),
exchange(REMOTE, GATHER, ImmutableList.of(),
Expand All @@ -156,9 +155,7 @@ public void testWithSortedTable()
public void testWithSortedWindowFunction()
{
List<PlanMatchPattern.Ordering> orderBy = ImmutableList.of(sort("col_b", ASCENDING, LAST));
assertPlanWithValidation(
"SELECT col_b, COUNT(*) OVER (ORDER BY col_b) FROM table_a ORDER BY col_b LIMIT 5",
output(
assertDistributedPlan("SELECT col_b, COUNT(*) OVER (ORDER BY col_b) FROM table_a ORDER BY col_b LIMIT 5", output(
topN(5, orderBy, FINAL,
exchange(LOCAL, GATHER, ImmutableList.of(),
limit(
Expand All @@ -181,9 +178,7 @@ public void testWithSortedWindowFunction()
@Test
public void testWithConstantProperty()
{
assertPlanWithValidation(
"SELECT * FROM (VALUES (1), (1)) AS t (id) WHERE id = 1 ORDER BY 1 LIMIT 1",
output(
assertDistributedPlan("SELECT * FROM (VALUES (1), (1)) AS t (id) WHERE id = 1 ORDER BY 1 LIMIT 1", output(
topN(1, ImmutableList.of(sort("id", ASCENDING, LAST)), FINAL,
exchange(LOCAL, GATHER, ImmutableList.of(),
limit(1, ImmutableList.of(), true, ImmutableList.of("id"),
Expand All @@ -195,15 +190,23 @@ public void testWithConstantProperty()
ImmutableList.of(new LongLiteral("1"))))))))));
}

private void assertPlanWithValidation(@Language("SQL") String sql, PlanMatchPattern pattern)
@Test
public void testNestedField()
{
LocalQueryRunner queryRunner = getQueryRunner();
queryRunner.inTransaction(queryRunner.getDefaultSession(), transactionSession -> {
Plan actualPlan = queryRunner.createPlan(transactionSession, sql, OPTIMIZED_AND_VALIDATED, false, WarningCollector.NOOP, createPlanOptimizersStatsCollector());
PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), queryRunner.getFunctionManager(), queryRunner.getStatsCalculator(), actualPlan, pattern);
PlannerContext plannerContext = queryRunner.getPlannerContext();
new ValidateLimitWithPresortedInput().validate(actualPlan.getRoot(), transactionSession, plannerContext, createTestingTypeAnalyzer(plannerContext), actualPlan.getTypes(), WarningCollector.NOOP);
return null;
});
assertDistributedPlan(
"""
SELECT nested.k
FROM with_nested_field
WHERE nested.k = 1
ORDER BY nested.k
LIMIT 1
""",
output(
topN(1, ImmutableList.of(sort("k", ASCENDING, LAST)), FINAL,
anyTree(
limit(1, ImmutableList.of(), true, ImmutableList.of("k"),
project(ImmutableMap.of("k", expression("nested[1]")),
filter("nested[1] = 1",
tableScan("with_nested_field", ImmutableMap.of("nested", "nested")))))))));
}
}
Loading

0 comments on commit 47de746

Please sign in to comment.