Skip to content

Commit

Permalink
Add unit test for PruneValuesColumns
Browse files Browse the repository at this point in the history
Add a new unit test for PruneValuesColumns.
Extend ValuesMatcher to optionally validate rows, and adjust the
PlanMatchPattern.values() functions accordingly.  The previous approach
was odd, in that there was a ValuesMatcher for each alias to be created;
it's more direct to implement Matcher and create the aliases in
detailMatches().
  • Loading branch information
alandpost authored and martint committed May 22, 2017
1 parent 7110b1b commit 05dcee1
Show file tree
Hide file tree
Showing 3 changed files with 156 additions and 21 deletions.
Expand Up @@ -47,12 +47,14 @@
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;

import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.IntStream;

import static com.facebook.presto.sql.ExpressionUtils.rewriteIdentifiersToSymbolReferences;
import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH;
Expand Down Expand Up @@ -309,28 +311,41 @@ public static PlanMatchPattern groupingSet(List<List<String>> groups, String gro
return node(GroupIdNode.class, source).with(new GroupIdMatcher(groups, ImmutableMap.of(), groupIdAlias));
}

private static PlanMatchPattern values(
Map<String, Integer> aliasToIndex,
Optional<Integer> expectedOutputSymbolCount,
Optional<List<List<Expression>>> expectedRows
)
{
return node(ValuesNode.class).with(new ValuesMatcher(aliasToIndex, expectedOutputSymbolCount, expectedRows));
}

private static PlanMatchPattern values(List<String> aliases, Optional<List<List<Expression>>> expectedRows)
{
return values(
Maps.uniqueIndex(IntStream.range(0, aliases.size()).boxed().iterator(), aliases::get),
Optional.of(aliases.size()),
expectedRows);
}

public static PlanMatchPattern values(Map<String, Integer> aliasToIndex)
{
PlanMatchPattern result = node(ValuesNode.class);
aliasToIndex.entrySet().forEach(
aliasWithIndex -> result.withAlias(aliasWithIndex.getKey(), new ValuesMatcher(aliasWithIndex.getValue())));
return result;
return values(aliasToIndex, Optional.empty(), Optional.empty());
}

public static PlanMatchPattern values(String ... aliases)
{
return values(ImmutableList.copyOf(aliases));
}

public static PlanMatchPattern values(List<String> aliases, List<List<Expression>> expectedRows)
{
return values(aliases, Optional.of(expectedRows));
}

public static PlanMatchPattern values(List<String> aliases)
{
PlanMatchPattern result = node(ValuesNode.class).withNumberOfOutputColumns(aliases.size());
int index = 0;
for (String alias : aliases) {
result.withAlias(alias, new ValuesMatcher(index));
++index;
}
return result;
return values(aliases, Optional.empty());
}

public static PlanMatchPattern limit(long limit, PlanMatchPattern source)
Expand Down
Expand Up @@ -14,32 +14,70 @@
package com.facebook.presto.sql.planner.assertions;

import com.facebook.presto.Session;
import com.facebook.presto.cost.PlanNodeCost;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.sql.planner.Symbol;
import com.facebook.presto.sql.planner.plan.PlanNode;
import com.facebook.presto.sql.planner.plan.ValuesNode;
import com.facebook.presto.sql.tree.Expression;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;

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

import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH;
import static com.facebook.presto.sql.planner.assertions.MatchResult.match;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkState;
import static java.util.Objects.requireNonNull;

public class ValuesMatcher
implements RvalueMatcher
implements Matcher
{
int outputIndex;
private final Map<String, Integer> outputSymbolAliases;
private final Optional<Integer> expectedOutputSymbolCount;
private final Optional<List<List<Expression>>> expectedRows;

public ValuesMatcher(int outputIndex)
public ValuesMatcher(
Map<String, Integer> outputSymbolAliases,
Optional<Integer> expectedOutputSymbolCount,
Optional<List<List<Expression>>> expectedRows)
{
this.outputIndex = outputIndex;
this.outputSymbolAliases = ImmutableMap.copyOf(outputSymbolAliases);
this.expectedOutputSymbolCount = requireNonNull(expectedOutputSymbolCount, "expectedOutputSymbolCount is null");
this.expectedRows = requireNonNull(expectedRows, "expectedRows is null");
}

@Override
public Optional<Symbol> getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases)
public boolean shapeMatches(PlanNode node)
{
if (!(node instanceof ValuesNode)) {
return Optional.empty();
}
return (node instanceof ValuesNode) &&
expectedOutputSymbolCount.map(Integer.valueOf(node.getOutputSymbols().size())::equals).orElse(true);
}

@Override
public MatchResult detailMatches(PlanNode node, PlanNodeCost planNodeCost, Session session, Metadata metadata, SymbolAliases symbolAliases)
{
checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName());
ValuesNode valuesNode = (ValuesNode) node;

return Optional.of(valuesNode.getOutputSymbols().get(outputIndex));
if (!expectedRows.map(rows -> rows.equals(valuesNode.getRows())).orElse(true)) {
return NO_MATCH;
}

return match(SymbolAliases.builder()
.putAll(Maps.transformValues(outputSymbolAliases, index -> valuesNode.getOutputSymbols().get(index).toSymbolReference()))
.build());
}

@Override
public String toString()
{
return toStringHelper(this)
.add("outputSymbolAliases", outputSymbolAliases)
.add("expectedOutputSymbolCount", expectedOutputSymbolCount)
.add("expectedRows", expectedRows)
.toString();
}
}
@@ -0,0 +1,82 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.sql.planner.iterative.rule;

import com.facebook.presto.sql.planner.assertions.PlanMatchPattern;
import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester;
import com.facebook.presto.sql.planner.plan.Assignments;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.values;
import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.expression;
import static io.airlift.testing.Closeables.closeAllRuntimeException;

public class TestPruneValuesColumns
{
private RuleTester tester;

@BeforeClass
public void setUp()
{
tester = new RuleTester();
}

@AfterClass(alwaysRun = true)
public void tearDown()
{
closeAllRuntimeException(tester);
tester = null;
}

@Test
public void testNotAllOutputsReferenced()
throws Exception
{
tester.assertThat(new PruneValuesColumns())
.on(p ->
p.project(
Assignments.of(p.symbol("y", BIGINT), expression("x")),
p.values(
ImmutableList.of(p.symbol("unused", BIGINT), p.symbol("x", BIGINT)),
ImmutableList.of(
ImmutableList.of(expression("1"), expression("2")),
ImmutableList.of(expression("3"), expression("4"))))))
.matches(
project(
ImmutableMap.of("y", PlanMatchPattern.expression("x")),
values(
ImmutableList.of("x"),
ImmutableList.of(
ImmutableList.of(expression("2")),
ImmutableList.of(expression("4"))))));
}

@Test
public void testAllOutputsReferenced()
throws Exception
{
tester.assertThat(new PruneValuesColumns())
.on(p ->
p.project(
Assignments.of(p.symbol("y", BIGINT), expression("x")),
p.values(p.symbol("x", BIGINT))))
.doesNotFire();
}
}

0 comments on commit 05dcee1

Please sign in to comment.