Skip to content

Commit

Permalink
Add PruneFilterColumns rule
Browse files Browse the repository at this point in the history
Migrate PruneUnreferencedOutputs handling of FilterNode to
the iterative optimizer.  This rule finds a filter that's under
a project, such that the grandchild produces columns not needed by
either the parent or the child, and creates a new project grandchild to
discard the unused columns.  Subsequent rules may match the pattern of
the new grandchild (project) over the old grandchild, and do further
pruning.
  • Loading branch information
alandpost authored and kokosing committed Jul 26, 2017
1 parent f875252 commit 64e5373
Show file tree
Hide file tree
Showing 3 changed files with 130 additions and 0 deletions.
Expand Up @@ -35,6 +35,7 @@
import com.facebook.presto.sql.planner.iterative.rule.MergeLimits;
import com.facebook.presto.sql.planner.iterative.rule.PruneCountAggregationOverScalar;
import com.facebook.presto.sql.planner.iterative.rule.PruneCrossJoinColumns;
import com.facebook.presto.sql.planner.iterative.rule.PruneFilterColumns;
import com.facebook.presto.sql.planner.iterative.rule.PruneIndexSourceColumns;
import com.facebook.presto.sql.planner.iterative.rule.PruneJoinChildrenColumns;
import com.facebook.presto.sql.planner.iterative.rule.PruneJoinColumns;
Expand Down Expand Up @@ -140,6 +141,7 @@ public PlanOptimizers(Metadata metadata, SqlParser sqlParser, FeaturesConfig fea
// TODO: Once we've migrated handling all the plan node types, replace uses of PruneUnreferencedOutputs with an IterativeOptimizer containing these rules.
Set<Rule> columnPruningRules = ImmutableSet.of(
new PruneCrossJoinColumns(),
new PruneFilterColumns(),
new PruneIndexSourceColumns(),
new PruneJoinChildrenColumns(),
new PruneJoinColumns(),
Expand Down
@@ -0,0 +1,47 @@
/*
* 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.PlanNodeIdAllocator;
import com.facebook.presto.sql.planner.Symbol;
import com.facebook.presto.sql.planner.SymbolsExtractor;
import com.facebook.presto.sql.planner.plan.FilterNode;
import com.facebook.presto.sql.planner.plan.PlanNode;
import com.google.common.collect.Streams;

import java.util.Optional;
import java.util.Set;

import static com.facebook.presto.sql.planner.iterative.rule.Util.restrictChildOutputs;
import static com.google.common.collect.ImmutableSet.toImmutableSet;

public class PruneFilterColumns
extends ProjectOffPushDownRule<FilterNode>
{
public PruneFilterColumns()
{
super(FilterNode.class);
}

@Override
protected Optional<PlanNode> pushDownProjectOff(PlanNodeIdAllocator idAllocator, FilterNode filterNode, Set<Symbol> referencedOutputs)
{
Set<Symbol> prunedFilterInputs = Streams.concat(
referencedOutputs.stream(),
SymbolsExtractor.extractUnique(filterNode.getPredicate()).stream())
.collect(toImmutableSet());

return restrictChildOutputs(idAllocator, filterNode, prunedFilterInputs);
}
}
@@ -0,0 +1,81 @@
/*
* 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.Symbol;
import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest;
import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder;
import com.facebook.presto.sql.planner.plan.Assignments;
import com.facebook.presto.sql.planner.plan.ProjectNode;
import com.google.common.collect.ImmutableMap;
import org.testng.annotations.Test;

import java.util.function.Predicate;
import java.util.stream.Stream;

import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.expression;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.filter;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.strictProject;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.values;
import static com.google.common.base.Predicates.alwaysTrue;
import static com.google.common.collect.ImmutableSet.toImmutableSet;

public class TestPruneFilterColumns
extends BaseRuleTest
{
@Test
public void testNotAllInputsReferenced()
throws Exception
{
tester().assertThat(new PruneFilterColumns())
.on(p -> buildProjectedFilter(p, symbol -> symbol.getName().equals("b")))
.matches(
strictProject(
ImmutableMap.of("b", expression("b")),
filter(
"b > 5",
strictProject(
ImmutableMap.of("b", expression("b")),
values("a", "b")))));
}

@Test
public void testAllInputsReferenced()
throws Exception
{
tester().assertThat(new PruneFilterColumns())
.on(p -> buildProjectedFilter(p, symbol -> symbol.getName().equals("a")))
.doesNotFire();
}

@Test
public void testAllOutputsReferenced()
throws Exception
{
tester().assertThat(new PruneFilterColumns())
.on(p -> buildProjectedFilter(p, alwaysTrue()))
.doesNotFire();
}

private ProjectNode buildProjectedFilter(PlanBuilder planBuilder, Predicate<Symbol> projectionFilter)
{
Symbol a = planBuilder.symbol("a");
Symbol b = planBuilder.symbol("b");
return planBuilder.project(
Assignments.identity(Stream.of(a, b).filter(projectionFilter).collect(toImmutableSet())),
planBuilder.filter(
planBuilder.expression("b > 5"),
planBuilder.values(a, b)));
}
}

0 comments on commit 64e5373

Please sign in to comment.