Skip to content

Commit

Permalink
Rename "cumulative cost" to "cost"
Browse files Browse the repository at this point in the history
The distinction is no longer relevant.

Extracted-From: https://github.com/prestodb/presto
  • Loading branch information
findepi authored and wenleix committed Mar 2, 2019
1 parent 0383c22 commit 6332099
Show file tree
Hide file tree
Showing 8 changed files with 43 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public CachingCostProvider(CostCalculator costCalculator, StatsProvider statsPro
}

@Override
public PlanCostEstimate getCumulativeCost(PlanNode node)
public PlanCostEstimate getCost(PlanNode node)
{
if (!isEnableStatsCalculator(session)) {
return PlanCostEstimate.unknown();
Expand All @@ -70,14 +70,14 @@ public PlanCostEstimate getCumulativeCost(PlanNode node)
return getGroupCost((GroupReference) node);
}

PlanCostEstimate cumulativeCost = cache.get(node);
if (cumulativeCost != null) {
return cumulativeCost;
PlanCostEstimate cost = cache.get(node);
if (cost != null) {
return cost;
}

cumulativeCost = calculateCumulativeCost(node);
verify(cache.put(node, cumulativeCost) == null, "Cost already set");
return cumulativeCost;
cost = calculateCost(node);
verify(cache.put(node, cost) == null, "Cost already set");
return cost;
}
catch (RuntimeException e) {
if (isIgnoreStatsCalculatorFailures(session)) {
Expand All @@ -93,18 +93,18 @@ private PlanCostEstimate getGroupCost(GroupReference groupReference)
int group = groupReference.getGroupId();
Memo memo = this.memo.orElseThrow(() -> new IllegalStateException("CachingCostProvider without memo cannot handle GroupReferences"));

Optional<PlanCostEstimate> cost = memo.getCumulativeCost(group);
if (cost.isPresent()) {
return cost.get();
Optional<PlanCostEstimate> knownCost = memo.getCost(group);
if (knownCost.isPresent()) {
return knownCost.get();
}

PlanCostEstimate cumulativeCost = calculateCumulativeCost(memo.getNode(group));
verify(!memo.getCumulativeCost(group).isPresent(), "Group cost already set");
memo.storeCumulativeCost(group, cumulativeCost);
return cumulativeCost;
PlanCostEstimate cost = calculateCost(memo.getNode(group));
verify(!memo.getCost(group).isPresent(), "Group cost already set");
memo.storeCost(group, cost);
return cost;
}

private PlanCostEstimate calculateCumulativeCost(PlanNode node)
private PlanCostEstimate calculateCost(PlanNode node)
{
return costCalculator.calculateCost(node, statsProvider, this, session, types);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public PlanCostEstimate calculateCost(PlanNode node, StatsProvider stats, CostPr
LocalCostEstimate localCost = node.accept(costEstimator, null);

PlanCostEstimate sourcesCost = node.getSources().stream()
.map(sourcesCosts::getCumulativeCost)
.map(sourcesCosts::getCost)
.reduce(PlanCostEstimate.zero(), CostCalculatorUsingExchanges::add);
return add(sourcesCost, localCost);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,5 +17,5 @@

public interface CostProvider
{
PlanCostEstimate getCumulativeCost(PlanNode node);
PlanCostEstimate getCost(PlanNode node);
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ public static StatsAndCosts create(PlanNode root, StatsProvider statsProvider, C
ImmutableMap.Builder<PlanNodeId, PlanCostEstimate> costs = ImmutableMap.builder();
for (PlanNode node : planIterator) {
stats.put(node.getId(), statsProvider.getStats(node));
costs.put(node.getId(), costProvider.getCumulativeCost(node));
costs.put(node.getId(), costProvider.getCost(node));
}
return new StatsAndCosts(stats.build(), costs.build());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ public PlanNode replace(int group, PlanNode node, String reason)
private void evictStatisticsAndCost(int group)
{
getGroup(group).stats = null;
getGroup(group).cumulativeCost = null;
getGroup(group).cost = null;
for (int parentGroup : getGroup(group).incomingReferences.elementSet()) {
if (parentGroup != ROOT_GROUP_REF) {
evictStatisticsAndCost(parentGroup);
Expand All @@ -160,14 +160,14 @@ public void storeStats(int groupId, PlanNodeStatsEstimate stats)
group.stats = requireNonNull(stats, "stats is null");
}

public Optional<PlanCostEstimate> getCumulativeCost(int group)
public Optional<PlanCostEstimate> getCost(int group)
{
return Optional.ofNullable(getGroup(group).cumulativeCost);
return Optional.ofNullable(getGroup(group).cost);
}

public void storeCumulativeCost(int group, PlanCostEstimate cost)
public void storeCost(int group, PlanCostEstimate cost)
{
getGroup(group).cumulativeCost = requireNonNull(cost, "cost is null");
getGroup(group).cost = requireNonNull(cost, "cost is null");
}

private void incrementReferenceCounts(PlanNode fromNode, int fromGroup)
Expand Down Expand Up @@ -256,7 +256,7 @@ static Group withMember(PlanNode member)
@Nullable
private PlanNodeStatsEstimate stats;
@Nullable
private PlanCostEstimate cumulativeCost;
private PlanCostEstimate cost;

private Group(PlanNode member)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -412,7 +412,7 @@ private List<JoinEnumerationResult> getPossibleJoinNodes(JoinNode joinNode, Dist

private JoinEnumerationResult createJoinEnumerationResult(PlanNode planNode)
{
return JoinEnumerationResult.createJoinEnumerationResult(Optional.of(planNode), costProvider.getCumulativeCost(planNode));
return JoinEnumerationResult.createJoinEnumerationResult(Optional.of(planNode), costProvider.getCost(planNode));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -531,7 +531,7 @@ private TestingCostProvider(Map<String, PlanCostEstimate> costs, CostCalculator
}

@Override
public PlanCostEstimate getCumulativeCost(PlanNode node)
public PlanCostEstimate getCost(PlanNode node)
{
if (costs.containsKey(node.getId().toString())) {
return costs.get(node.getId().toString());
Expand All @@ -548,13 +548,13 @@ private CostAssertionBuilder assertCost(
Map<String, Type> types)
{
Function<PlanNode, PlanNodeStatsEstimate> statsProvider = planNode -> stats.get(planNode.getId().toString());
PlanCostEstimate cumulativeCost = calculateCumulativeCost(
PlanCostEstimate cost = calculateCost(
costCalculator,
node,
sourceCostProvider(costCalculator, costs, statsProvider, types),
statsProvider,
types);
return new CostAssertionBuilder(cumulativeCost);
return new CostAssertionBuilder(cost);
}

private Function<PlanNode, PlanCostEstimate> sourceCostProvider(
Expand All @@ -568,7 +568,7 @@ private Function<PlanNode, PlanCostEstimate> sourceCostProvider(
if (providedCost != null) {
return providedCost;
}
return calculateCumulativeCost(
return calculateCost(
costCalculator,
node,
sourceCostProvider(costCalculator, costs, statsProvider, types),
Expand All @@ -579,14 +579,14 @@ private Function<PlanNode, PlanCostEstimate> sourceCostProvider(

private void assertCostHasUnknownComponentsForUnknownStats(PlanNode node, Map<String, Type> types)
{
new CostAssertionBuilder(calculateCumulativeCost(
new CostAssertionBuilder(calculateCost(
costCalculatorUsingExchanges,
node,
planNode -> PlanCostEstimate.unknown(),
planNode -> PlanNodeStatsEstimate.unknown(),
types))
.hasUnknownComponents();
new CostAssertionBuilder(calculateCumulativeCost(
new CostAssertionBuilder(calculateCost(
costCalculatorWithEstimatedExchanges,
node,
planNode -> PlanCostEstimate.unknown(),
Expand All @@ -598,8 +598,8 @@ private void assertCostHasUnknownComponentsForUnknownStats(PlanNode node, Map<St
private void assertFragmentedEqualsUnfragmented(PlanNode node, Map<String, PlanNodeStatsEstimate> stats, Map<String, Type> types)
{
StatsCalculator statsCalculator = statsCalculator(stats);
PlanCostEstimate costWithExchanges = calculateCumulativeCost(node, costCalculatorUsingExchanges, statsCalculator, types);
PlanCostEstimate costWithFragments = calculateCumulativeCostFragmentedPlan(node, statsCalculator, types);
PlanCostEstimate costWithExchanges = calculateCost(node, costCalculatorUsingExchanges, statsCalculator, types);
PlanCostEstimate costWithFragments = calculateCostFragmentedPlan(node, statsCalculator, types);
assertEquals(costWithExchanges, costWithFragments);
}

Expand All @@ -609,7 +609,7 @@ private StatsCalculator statsCalculator(Map<String, PlanNodeStatsEstimate> stats
requireNonNull(stats.get(node.getId().toString()), "no stats for node");
}

private PlanCostEstimate calculateCumulativeCost(
private PlanCostEstimate calculateCost(
CostCalculator costCalculator,
PlanNode node,
Function<PlanNode, PlanCostEstimate> costs,
Expand All @@ -625,16 +625,16 @@ private PlanCostEstimate calculateCumulativeCost(
.collect(ImmutableMap.toImmutableMap(entry -> new Symbol(entry.getKey()), Map.Entry::getValue))));
}

private PlanCostEstimate calculateCumulativeCost(PlanNode node, CostCalculator costCalculator, StatsCalculator statsCalculator, Map<String, Type> types)
private PlanCostEstimate calculateCost(PlanNode node, CostCalculator costCalculator, StatsCalculator statsCalculator, Map<String, Type> types)
{
TypeProvider typeProvider = TypeProvider.copyOf(types.entrySet().stream()
.collect(ImmutableMap.toImmutableMap(entry -> new Symbol(entry.getKey()), Map.Entry::getValue)));
StatsProvider statsProvider = new CachingStatsProvider(statsCalculator, session, typeProvider);
CostProvider costProvider = new CachingCostProvider(costCalculator, statsProvider, Optional.empty(), session, typeProvider);
return costProvider.getCumulativeCost(node);
return costProvider.getCost(node);
}

private PlanCostEstimate calculateCumulativeCostFragmentedPlan(PlanNode node, StatsCalculator statsCalculator, Map<String, Type> types)
private PlanCostEstimate calculateCostFragmentedPlan(PlanNode node, StatsCalculator statsCalculator, Map<String, Type> types)
{
TypeProvider typeProvider = TypeProvider.copyOf(types.entrySet().stream()
.collect(ImmutableMap.toImmutableMap(entry -> new Symbol(entry.getKey()), Map.Entry::getValue)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -241,16 +241,16 @@ public void testEvictCostOnReplace()
PlanCostEstimate yCost = new PlanCostEstimate(42, 0, 0);
PlanCostEstimate xCost = new PlanCostEstimate(42, 0, 37);

memo.storeCumulativeCost(yGroup, yCost);
memo.storeCumulativeCost(xGroup, xCost);
memo.storeCost(yGroup, yCost);
memo.storeCost(xGroup, xCost);

assertEquals(memo.getCumulativeCost(yGroup), Optional.of(yCost));
assertEquals(memo.getCumulativeCost(xGroup), Optional.of(xCost));
assertEquals(memo.getCost(yGroup), Optional.of(yCost));
assertEquals(memo.getCost(xGroup), Optional.of(xCost));

memo.replace(yGroup, node(), "rule");

assertEquals(memo.getCumulativeCost(yGroup), Optional.empty());
assertEquals(memo.getCumulativeCost(xGroup), Optional.empty());
assertEquals(memo.getCost(yGroup), Optional.empty());
assertEquals(memo.getCost(xGroup), Optional.empty());
}

private static void assertMatchesStructure(PlanNode actual, PlanNode expected)
Expand Down

0 comments on commit 6332099

Please sign in to comment.