Skip to content

Commit

Permalink
Lookup.resolveGroup returns a Stream instead of List
Browse files Browse the repository at this point in the history
Stream doesn't enforce any materialization of plan nodes
  • Loading branch information
sopel39 committed Aug 28, 2017
1 parent 93e0b47 commit 40f7aa9
Show file tree
Hide file tree
Showing 4 changed files with 13 additions and 13 deletions.
Expand Up @@ -33,6 +33,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Stream;

import static com.facebook.presto.spi.StandardErrorCode.OPTIMIZER_TIMEOUT;
import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -76,7 +77,7 @@ public PlanNode optimize(PlanNode plan, Session session, Map<Symbol, Type> types
}

Memo memo = new Memo(idAllocator, plan);
Lookup lookup = Lookup.from(planNode -> ImmutableList.of(memo.resolve(planNode)));
Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode)));
Matcher matcher = new PlanNodeMatcher(lookup);

Duration timeout = SystemSessionProperties.getOptimizerTimeout(session);
Expand Down
Expand Up @@ -14,13 +14,12 @@
package com.facebook.presto.sql.planner.iterative;

import com.facebook.presto.sql.planner.plan.PlanNode;
import com.google.common.collect.ImmutableList;

import java.util.List;
import java.util.function.Function;
import java.util.stream.Stream;

import static com.google.common.base.Verify.verify;
import static com.google.common.collect.Iterables.getOnlyElement;
import static com.google.common.collect.MoreCollectors.toOptional;

public interface Lookup
{
Expand All @@ -35,7 +34,7 @@ public interface Lookup
@Deprecated
default PlanNode resolve(PlanNode node)
{
return getOnlyElement(resolveGroup(node));
return resolveGroup(node).collect(toOptional()).get();
}

/**
Expand All @@ -45,7 +44,7 @@ default PlanNode resolve(PlanNode node)
* If the node is not a GroupReference, it returns the
* singleton of the argument node.
*/
List<PlanNode> resolveGroup(PlanNode node);
Stream<PlanNode> resolveGroup(PlanNode node);

/**
* A Lookup implementation that does not perform lookup. It satisfies contract
Expand All @@ -55,18 +54,18 @@ static Lookup noLookup()
{
return node -> {
verify(!(node instanceof GroupReference), "Unexpected GroupReference");
return ImmutableList.of(node);
return Stream.of(node);
};
}

static Lookup from(Function<GroupReference, List<PlanNode>> resolver)
static Lookup from(Function<GroupReference, Stream<PlanNode>> resolver)
{
return node -> {
if (node instanceof GroupReference) {
return resolver.apply((GroupReference) node);
}

return ImmutableList.of(node);
return Stream.of(node);
};
}
}
Expand Up @@ -15,13 +15,13 @@

import com.facebook.presto.sql.planner.PlanNodeIdAllocator;
import com.facebook.presto.sql.planner.plan.PlanNode;
import com.google.common.collect.ImmutableList;

import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static com.facebook.presto.sql.planner.iterative.Plans.resolveGroupReferences;
import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -93,7 +93,7 @@ public PlanNode extract()

private PlanNode extract(PlanNode node)
{
return resolveGroupReferences(node, Lookup.from(planNode -> ImmutableList.of(this.resolve(planNode))));
return resolveGroupReferences(node, Lookup.from(planNode -> Stream.of(this.resolve(planNode))));
}

public PlanNode replace(int group, PlanNode node, String reason)
Expand Down
Expand Up @@ -33,12 +33,12 @@
import com.facebook.presto.sql.planner.plan.PlanNodeId;
import com.facebook.presto.sql.planner.planPrinter.PlanPrinter;
import com.facebook.presto.transaction.TransactionManager;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;

import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Stream;

import static com.facebook.presto.sql.planner.assertions.PlanAssert.assertPlan;
import static com.facebook.presto.transaction.TransactionBuilder.transaction;
Expand Down Expand Up @@ -147,7 +147,7 @@ private RuleApplication applyRule()
{
SymbolAllocator symbolAllocator = new SymbolAllocator(symbols);
Memo memo = new Memo(idAllocator, plan);
Lookup lookup = Lookup.from(planNode -> ImmutableList.of(memo.resolve(planNode)));
Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode)));

PlanNode memoRoot = memo.getNode(memo.getRootGroup());

Expand Down

0 comments on commit 40f7aa9

Please sign in to comment.