Skip to content

Commit

Permalink
Add test for PruneTableScanColumns
Browse files Browse the repository at this point in the history
Includes a PlanBuilder.tableScan() overload that provides the
TableHandle.  That overload is lifted from not-yet-merged 9cbe62cb,
which implemnents a PushDownTableConstraints rule.  The change is
identical, so later merging or rebasing should go smoothly.
  • Loading branch information
alandpost authored and martint committed May 22, 2017
1 parent 6347a66 commit 0e9b75a
Show file tree
Hide file tree
Showing 2 changed files with 101 additions and 3 deletions.
@@ -0,0 +1,94 @@
/*
* 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.connector.ConnectorId;
import com.facebook.presto.metadata.TableHandle;
import com.facebook.presto.sql.planner.Symbol;
import com.facebook.presto.sql.planner.TestingColumnHandle;
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.facebook.presto.tpch.TpchColumnHandle;
import com.facebook.presto.tpch.TpchTableHandle;
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.spi.type.DateType.DATE;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.strictProject;
import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.strictTableScan;
import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.expression;
import static com.facebook.presto.tpch.TpchMetadata.TINY_SCALE_FACTOR;
import static io.airlift.testing.Closeables.closeAllRuntimeException;

public class TestPruneTableScanColumns
{
private RuleTester tester;

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

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

@Test
public void testNotAllOutputsReferenced()
{
tester.assertThat(new PruneTableScanColumns())
.on(p ->
{
Symbol orderdate = p.symbol("orderdate", DATE);
Symbol totalprice = p.symbol("totalprice", DOUBLE);
return p.project(
Assignments.of(p.symbol("x", BIGINT), totalprice.toSymbolReference()),
p.tableScan(
new TableHandle(
new ConnectorId("local"),
new TpchTableHandle("local", "orders", TINY_SCALE_FACTOR)),
ImmutableList.of(orderdate, totalprice),
ImmutableMap.of(
orderdate, new TpchColumnHandle(orderdate.getName(), DATE),
totalprice, new TpchColumnHandle(totalprice.getName(), DOUBLE))));
})
.matches(
strictProject(
ImmutableMap.of("x_", PlanMatchPattern.expression("totalprice_")),
strictTableScan("orders", ImmutableMap.of("totalprice_", "totalprice"))));
}

@Test
public void testAllOutputsReferenced()
{
tester.assertThat(new PruneTableScanColumns())
.on(p ->
p.project(
Assignments.of(p.symbol("y", BIGINT), expression("x")),
p.tableScan(
ImmutableList.of(p.symbol("x", BIGINT)),
ImmutableMap.of(p.symbol("x", BIGINT), new TestingColumnHandle("x")))))
.doesNotFire();
}
}
Expand Up @@ -200,13 +200,17 @@ public ApplyNode apply(Assignments subqueryAssignments, List<Symbol> correlation
}

public TableScanNode tableScan(List<Symbol> symbols, Map<Symbol, ColumnHandle> assignments)
{
TableHandle tableHandle = new TableHandle(new ConnectorId("testConnector"), new TestingTableHandle());
return tableScan(tableHandle, symbols, assignments);
}

public TableScanNode tableScan(TableHandle tableHandle, List<Symbol> symbols, Map<Symbol, ColumnHandle> assignments)
{
Expression originalConstraint = null;
return new TableScanNode(
idAllocator.getNextId(),
new TableHandle(
new ConnectorId("testConnector"),
new TestingTableHandle()),
tableHandle,
symbols,
assignments,
Optional.empty(),
Expand Down

0 comments on commit 0e9b75a

Please sign in to comment.