Skip to content

Commit

Permalink
Close drivers and operators in operator tests
Browse files Browse the repository at this point in the history
  • Loading branch information
pnowojski authored and haozhun committed Sep 30, 2016
1 parent 2b3f0e1 commit 6641d58
Show file tree
Hide file tree
Showing 18 changed files with 187 additions and 284 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.facebook.presto.spi.block.InterleavedBlockBuilder;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.testing.MaterializedResult;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;

import java.util.ArrayList;
Expand Down Expand Up @@ -95,7 +96,17 @@ else if (!finishing) {
return outputPages.build();
}

public static List<Page> toPages(Operator operator, List<Page> input)
public static List<Page> toPages(OperatorFactory operatorFactory, DriverContext driverContext, List<Page> input)
{
try (Operator operator = operatorFactory.createOperator(driverContext)) {
return toPages(operator, input);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}

private static List<Page> toPages(Operator operator, List<Page> input)
{
// verify initial state
assertEquals(operator.isFinished(), false);
Expand All @@ -105,7 +116,17 @@ public static List<Page> toPages(Operator operator, List<Page> input)
return toPages(operator, input.iterator());
}

public static List<Page> toPages(Operator operator)
public static List<Page> toPages(OperatorFactory operatorFactory, DriverContext driverContext)
{
try (Operator operator = operatorFactory.createOperator(driverContext)) {
return toPages(operator);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}

private static List<Page> toPages(Operator operator)
{
// operator does not have input so should never require input
assertEquals(operator.needsInput(), false);
Expand Down Expand Up @@ -155,62 +176,66 @@ public static Block toRow(List<Type> parameterTypes, Object... values)
return blockBuilder.build();
}

public static void assertOperatorEquals(Operator operator, List<Page> expected)
{
List<Page> actual = toPages(operator);
assertEquals(actual.size(), expected.size());
for (int i = 0; i < actual.size(); i++) {
assertPageEquals(operator.getTypes(), actual.get(i), expected.get(i));
}
}

public static void assertOperatorEquals(Operator operator, List<Page> input, List<Page> expected)
public static void assertOperatorEquals(OperatorFactory operatorFactory, DriverContext driverContext, List<Page> input, List<Page> expected)
throws Exception
{
List<Page> actual = toPages(operator, input);
List<Page> actual = toPages(operatorFactory, driverContext, input);
assertEquals(actual.size(), expected.size());
for (int i = 0; i < actual.size(); i++) {
assertPageEquals(operator.getTypes(), actual.get(i), expected.get(i));
assertPageEquals(operatorFactory.getTypes(), actual.get(i), expected.get(i));
}
}

public static void assertOperatorEquals(Operator operator, List<Page> input, MaterializedResult expected)
public static void assertOperatorEquals(OperatorFactory operatorFactory, DriverContext driverContext, List<Page> input, MaterializedResult expected)
throws Exception
{
assertOperatorEquals(operator, input, expected, false, ImmutableList.<Integer>of());
assertOperatorEquals(operatorFactory, driverContext, input, expected, false, ImmutableList.of());
}

public static void assertOperatorEquals(Operator operator, List<Page> input, MaterializedResult expected, boolean hashEnabled, List<Integer> hashChannels)
public static void assertOperatorEquals(OperatorFactory operatorFactory, DriverContext driverContext, List<Page> input, MaterializedResult expected, boolean hashEnabled, List<Integer> hashChannels)
throws Exception
{
List<Page> pages = toPages(operator, input);
List<Page> pages = toPages(operatorFactory, driverContext, input);
MaterializedResult actual;
if (hashEnabled && !hashChannels.isEmpty()) {
// Drop the hashChannel for all pages
List<Page> actualPages = dropChannel(pages, hashChannels);
List<Type> expectedTypes = without(operator.getTypes(), hashChannels);
actual = toMaterializedResult(operator.getOperatorContext().getSession(), expectedTypes, actualPages);
List<Type> expectedTypes = without(operatorFactory.getTypes(), hashChannels);
actual = toMaterializedResult(driverContext.getSession(), expectedTypes, actualPages);
}
else {
actual = toMaterializedResult(operator.getOperatorContext().getSession(), operator.getTypes(), pages);
actual = toMaterializedResult(driverContext.getSession(), operatorFactory.getTypes(), pages);
}
assertEquals(actual, expected);
}

public static void assertOperatorEqualsIgnoreOrder(Operator operator, List<Page> input, MaterializedResult expected)
public static void assertOperatorEqualsIgnoreOrder(
OperatorFactory operatorFactory,
DriverContext driverContext,
List<Page> input,
MaterializedResult expected)
{
assertOperatorEqualsIgnoreOrder(operator, input, expected, false, Optional.empty());
assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, false, Optional.empty());
}

public static void assertOperatorEqualsIgnoreOrder(Operator operator, List<Page> input, MaterializedResult expected, boolean hashEnabled, Optional<Integer> hashChannel)
public static void assertOperatorEqualsIgnoreOrder(
OperatorFactory operatorFactory,
DriverContext driverContext,
List<Page> input,
MaterializedResult expected,
boolean hashEnabled,
Optional<Integer> hashChannel)
{
List<Page> pages = toPages(operator, input);
List<Page> pages = toPages(operatorFactory, driverContext, input);
MaterializedResult actual;
if (hashEnabled && hashChannel.isPresent()) {
// Drop the hashChannel for all pages
List<Page> actualPages = dropChannel(pages, ImmutableList.of(hashChannel.get()));
List<Type> expectedTypes = without(operator.getTypes(), ImmutableList.of(hashChannel.get()));
actual = toMaterializedResult(operator.getOperatorContext().getSession(), expectedTypes, actualPages);
List<Type> expectedTypes = without(operatorFactory.getTypes(), ImmutableList.of(hashChannel.get()));
actual = toMaterializedResult(driverContext.getSession(), expectedTypes, actualPages);
}
else {
actual = toMaterializedResult(operator.getOperatorContext().getSession(), operator.getTypes(), pages);
actual = toMaterializedResult(driverContext.getSession(), operatorFactory.getTypes(), pages);
}

assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,12 +106,11 @@ public void testAggregation()
REAL_SUM.bind(ImmutableList.of(4), Optional.empty(), Optional.empty(), 1.0),
DOUBLE_SUM.bind(ImmutableList.of(5), Optional.empty(), Optional.empty(), 1.0),
maxVarcharColumn.bind(ImmutableList.of(6), Optional.empty(), Optional.empty(), 1.0)));
Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT, BIGINT, DOUBLE, VARCHAR, BIGINT, BIGINT, DOUBLE, VARCHAR)
.row(100L, 4950L, 49.5, "399", 100L, 54950L, 44950.0f, 54950.0, "599")
.build();

assertOperatorEquals(operator, input, expected);
assertOperatorEquals(operatorFactory, driverContext, input, expected);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import static com.facebook.presto.RowPagesBuilder.rowPagesBuilder;
import static com.facebook.presto.SessionTestUtils.TEST_SESSION;
import static com.facebook.presto.operator.OperatorAssertion.assertOperatorEquals;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.testing.MaterializedResult.resultBuilder;
import static com.facebook.presto.testing.TestingTaskContext.createTaskContext;
Expand Down Expand Up @@ -73,7 +74,6 @@ public void testDistinctLimit(boolean hashEnabled)
.build();

OperatorFactory operatorFactory = new DistinctLimitOperator.DistinctLimitOperatorFactory(0, new PlanNodeId("test"), ImmutableList.of(BIGINT), Ints.asList(0), 5, rowPagesBuilder.getHashChannel());
Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT)
.row(1L)
Expand All @@ -83,7 +83,7 @@ public void testDistinctLimit(boolean hashEnabled)
.row(5L)
.build();

OperatorAssertion.assertOperatorEquals(operator, input, expected);
assertOperatorEquals(operatorFactory, driverContext, input, expected);
}

@Test(dataProvider = "hashEnabledValues")
Expand All @@ -97,15 +97,14 @@ public void testDistinctLimitWithPageAlignment(boolean hashEnabled)
.build();

OperatorFactory operatorFactory = new DistinctLimitOperator.DistinctLimitOperatorFactory(0, new PlanNodeId("test"), ImmutableList.of(BIGINT), Ints.asList(0), 3, rowPagesBuilder.getHashChannel());
Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT)
.row(1L)
.row(2L)
.row(3L)
.build();

OperatorAssertion.assertOperatorEquals(operator, input, expected);
assertOperatorEquals(operatorFactory, driverContext, input, expected);
}

@Test(dataProvider = "hashEnabledValues")
Expand All @@ -119,7 +118,6 @@ public void testDistinctLimitValuesLessThanLimit(boolean hashEnabled)
.build();

OperatorFactory operatorFactory = new DistinctLimitOperator.DistinctLimitOperatorFactory(0, new PlanNodeId("test"), ImmutableList.of(BIGINT), Ints.asList(0), 5, rowPagesBuilder.getHashChannel());
Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT)
.row(1L)
Expand All @@ -128,6 +126,6 @@ public void testDistinctLimitValuesLessThanLimit(boolean hashEnabled)
.row(4L)
.build();

OperatorAssertion.assertOperatorEquals(operator, input, expected);
assertOperatorEquals(operatorFactory, driverContext, input, expected);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -99,8 +99,6 @@ public Set<Integer> getInputChannels()
() -> new GenericPageProcessor(filter, ImmutableList.of(singleColumn(VARCHAR, 0), new Add5Projection(1))),
ImmutableList.<Type>of(VARCHAR, BIGINT));

Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = MaterializedResult.resultBuilder(driverContext.getSession(), VARCHAR, BIGINT)
.row("10", 15L)
.row("11", 16L)
Expand All @@ -114,7 +112,7 @@ public Set<Integer> getInputChannels()
.row("19", 24L)
.build();

assertOperatorEquals(operator, input, expected);
assertOperatorEquals(operatorFactory, driverContext, input, expected);
}

private static class Add5Projection
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,15 +28,13 @@

import static com.facebook.presto.RowPagesBuilder.rowPagesBuilder;
import static com.facebook.presto.SessionTestUtils.TEST_SESSION;
import static com.facebook.presto.operator.OperatorAssertion.toMaterializedResult;
import static com.facebook.presto.operator.OperatorAssertion.toPages;
import static com.facebook.presto.operator.OperatorAssertion.assertOperatorEqualsIgnoreOrder;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.testing.MaterializedResult.resultBuilder;
import static com.facebook.presto.testing.TestingTaskContext.createTaskContext;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder;
import static java.util.concurrent.Executors.newCachedThreadPool;

@Test(singleThreaded = true)
Expand Down Expand Up @@ -78,8 +76,6 @@ public void testGroupId()
ImmutableList.of(1, 2, 3),
ImmutableList.of(0));

Operator operator = operatorFactory.createOperator(driverContext);

MaterializedResult expected = resultBuilder(driverContext.getSession(), VARCHAR, BOOLEAN, BIGINT, BIGINT, BIGINT)
.row("400", true, null, 100L, 0L)
.row("401", false, null, 101L, 0L)
Expand All @@ -95,9 +91,6 @@ public void testGroupId()
.row(null, null, 1102L, 202L, 1L)
.build();

List<Page> pages = toPages(operator, input.iterator());
MaterializedResult actual = toMaterializedResult(operator.getOperatorContext().getSession(), operator.getTypes(), pages);

assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows());
assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected);
}
}
Loading

0 comments on commit 6641d58

Please sign in to comment.