Skip to content

Commit

Permalink
Throw PrestoException from VisitPlan.
Browse files Browse the repository at this point in the history
  • Loading branch information
Amit Dutta authored and rschlussel committed Mar 24, 2022
1 parent 63a6b0f commit d6c9779
Show file tree
Hide file tree
Showing 2 changed files with 30 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,9 @@
*/
package com.facebook.presto.sql.planner.iterative.rule;

import com.facebook.presto.Session;
import com.facebook.presto.matching.Captures;
import com.facebook.presto.matching.Pattern;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.plan.FilterNode;
import com.facebook.presto.spi.plan.ProjectNode;
import com.facebook.presto.spi.relation.VariableReferenceExpression;
Expand All @@ -30,6 +30,7 @@

import static com.facebook.presto.SystemSessionProperties.isOffsetClauseEnabled;
import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.sql.analyzer.ExpressionTreeUtils.createSymbolReference;
import static com.facebook.presto.sql.planner.plan.AssignmentUtils.identityAssignmentsAsSymbolReferences;
import static com.facebook.presto.sql.planner.plan.Patterns.offset;
Expand Down Expand Up @@ -63,15 +64,13 @@ public Pattern<OffsetNode> getPattern()
return PATTERN;
}

@Override
public boolean isEnabled(Session session)
{
return isOffsetClauseEnabled(session);
}

@Override
public Result apply(OffsetNode parent, Captures captures, Context context)
{
if (!isOffsetClauseEnabled(context.getSession())) {
throw new PrestoException(NOT_SUPPORTED, "Offset support is not enabled");
}

VariableReferenceExpression rowNumberSymbol = context.getVariableAllocator().newVariable("row_number", BIGINT);

RowNumberNode rowNumberNode = new RowNumberNode(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.sql.planner.iterative.rule;

import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.relation.VariableReferenceExpression;
import com.facebook.presto.sql.planner.assertions.ExpressionMatcher;
import com.facebook.presto.sql.planner.assertions.RowNumberSymbolMatcher;
Expand Down Expand Up @@ -57,6 +58,29 @@ public void testReplaceOffsetOverValues()
.withAlias("row_num", new RowNumberSymbolMatcher()))));
}

@Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Offset support is not enabled")
public void testOffsetClauseDisabled()
{
tester().assertThat(new ImplementOffset())
.on(p -> {
VariableReferenceExpression a = p.variable("a");
VariableReferenceExpression b = p.variable("b");
return p.offset(
2,
p.values(a, b));
})
.matches(
strictProject(
ImmutableMap.of("a", new ExpressionMatcher("a"), "b", new ExpressionMatcher("b")),
filter(
"row_num > BIGINT '2'",
rowNumber(
pattern -> pattern
.partitionBy(ImmutableList.of()),
values("a", "b"))
.withAlias("row_num", new RowNumberSymbolMatcher()))));
}

@Test
public void testReplaceOffsetOverSort()
{
Expand Down

0 comments on commit d6c9779

Please sign in to comment.