Skip to content

Commit

Permalink
Change PlanPrinter to use FunctionRegistry instead of Metadata
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Mar 28, 2018
1 parent 7f4b4a5 commit d45566e
Show file tree
Hide file tree
Showing 8 changed files with 67 additions and 62 deletions.
Expand Up @@ -19,7 +19,7 @@
import com.facebook.presto.execution.QueryPerformanceFetcher; import com.facebook.presto.execution.QueryPerformanceFetcher;
import com.facebook.presto.execution.StageId; import com.facebook.presto.execution.StageId;
import com.facebook.presto.execution.StageInfo; import com.facebook.presto.execution.StageInfo;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.FunctionRegistry;
import com.facebook.presto.spi.Page; import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.block.BlockBuilderStatus;
Expand All @@ -44,7 +44,7 @@ public static class ExplainAnalyzeOperatorFactory
private final int operatorId; private final int operatorId;
private final PlanNodeId planNodeId; private final PlanNodeId planNodeId;
private final QueryPerformanceFetcher queryPerformanceFetcher; private final QueryPerformanceFetcher queryPerformanceFetcher;
private final Metadata metadata; private final FunctionRegistry functionRegistry;
private final StatsCalculator statsCalculator; private final StatsCalculator statsCalculator;
private final CostCalculator costCalculator; private final CostCalculator costCalculator;
private final boolean verbose; private final boolean verbose;
Expand All @@ -54,15 +54,15 @@ public ExplainAnalyzeOperatorFactory(
int operatorId, int operatorId,
PlanNodeId planNodeId, PlanNodeId planNodeId,
QueryPerformanceFetcher queryPerformanceFetcher, QueryPerformanceFetcher queryPerformanceFetcher,
Metadata metadata, FunctionRegistry functionRegistry,
StatsCalculator statsCalculator, StatsCalculator statsCalculator,
CostCalculator costCalculator, CostCalculator costCalculator,
boolean verbose) boolean verbose)
{ {
this.operatorId = operatorId; this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null"); this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null");
this.metadata = requireNonNull(metadata, "metadata is null"); this.functionRegistry = requireNonNull(functionRegistry, "functionRegistry is null");
this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null");
this.costCalculator = requireNonNull(costCalculator, "costCalculator is null"); this.costCalculator = requireNonNull(costCalculator, "costCalculator is null");
this.verbose = verbose; this.verbose = verbose;
Expand All @@ -79,7 +79,7 @@ public Operator createOperator(DriverContext driverContext)
{ {
checkState(!closed, "Factory is already closed"); checkState(!closed, "Factory is already closed");
OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, ExplainAnalyzeOperator.class.getSimpleName()); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, ExplainAnalyzeOperator.class.getSimpleName());
return new ExplainAnalyzeOperator(operatorContext, queryPerformanceFetcher, metadata, statsCalculator, costCalculator, verbose); return new ExplainAnalyzeOperator(operatorContext, queryPerformanceFetcher, functionRegistry, statsCalculator, costCalculator, verbose);
} }


@Override @Override
Expand All @@ -91,13 +91,13 @@ public void noMoreOperators()
@Override @Override
public OperatorFactory duplicate() public OperatorFactory duplicate()
{ {
return new ExplainAnalyzeOperatorFactory(operatorId, planNodeId, queryPerformanceFetcher, metadata, statsCalculator, costCalculator, verbose); return new ExplainAnalyzeOperatorFactory(operatorId, planNodeId, queryPerformanceFetcher, functionRegistry, statsCalculator, costCalculator, verbose);
} }
} }


private final OperatorContext operatorContext; private final OperatorContext operatorContext;
private final QueryPerformanceFetcher queryPerformanceFetcher; private final QueryPerformanceFetcher queryPerformanceFetcher;
private final Metadata metadata; private final FunctionRegistry functionRegistry;
private final StatsCalculator statsCalculator; private final StatsCalculator statsCalculator;
private final CostCalculator costCalculator; private final CostCalculator costCalculator;
private final boolean verbose; private final boolean verbose;
Expand All @@ -107,14 +107,14 @@ public OperatorFactory duplicate()
public ExplainAnalyzeOperator( public ExplainAnalyzeOperator(
OperatorContext operatorContext, OperatorContext operatorContext,
QueryPerformanceFetcher queryPerformanceFetcher, QueryPerformanceFetcher queryPerformanceFetcher,
Metadata metadata, FunctionRegistry functionRegistry,
StatsCalculator statsCalculator, StatsCalculator statsCalculator,
CostCalculator costCalculator, CostCalculator costCalculator,
boolean verbose) boolean verbose)
{ {
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null"); this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null");
this.metadata = requireNonNull(metadata, "metadata is null"); this.functionRegistry = requireNonNull(functionRegistry, "functionRegistry is null");
this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null");
this.costCalculator = requireNonNull(costCalculator, "costCalculator is null"); this.costCalculator = requireNonNull(costCalculator, "costCalculator is null");
this.verbose = verbose; this.verbose = verbose;
Expand Down Expand Up @@ -172,7 +172,7 @@ public Page getOutput()
return null; return null;
} }


String plan = textDistributedPlan(queryInfo.getOutputStage().get(), metadata, statsCalculator, costCalculator, operatorContext.getSession(), verbose); String plan = textDistributedPlan(queryInfo.getOutputStage().get(), functionRegistry, statsCalculator, costCalculator, operatorContext.getSession(), verbose);
BlockBuilder builder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1); BlockBuilder builder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1);
VARCHAR.writeString(builder, plan); VARCHAR.writeString(builder, plan);


Expand Down
Expand Up @@ -111,17 +111,17 @@ public String getPlan(Session session, Statement statement, Type planType, List<
switch (planType) { switch (planType) {
case LOGICAL: case LOGICAL:
Plan plan = getLogicalPlan(session, statement, parameters); Plan plan = getLogicalPlan(session, statement, parameters);
return PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata, statsCalculator, costCalculator, session); return PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionRegistry(), statsCalculator, costCalculator, session);
case DISTRIBUTED: case DISTRIBUTED:
SubPlan subPlan = getDistributedPlan(session, statement, parameters); SubPlan subPlan = getDistributedPlan(session, statement, parameters);
return PlanPrinter.textDistributedPlan(subPlan, metadata, statsCalculator, costCalculator, session); return PlanPrinter.textDistributedPlan(subPlan, metadata.getFunctionRegistry(), statsCalculator, costCalculator, session);
} }
throw new IllegalArgumentException("Unhandled plan type: " + planType); throw new IllegalArgumentException("Unhandled plan type: " + planType);
} }


public String getPlan(PlanFragment fragment, Session session) public String getPlan(PlanFragment fragment, Session session)
{ {
return PlanPrinter.textPlanFragment(fragment, metadata, statsCalculator, costCalculator, session); return PlanPrinter.textPlanFragment(fragment, metadata.getFunctionRegistry(), statsCalculator, costCalculator, session);
} }


private static <T extends Statement> String explainTask(Statement statement, DataDefinitionTask<T> task, List<Expression> parameters) private static <T extends Statement> String explainTask(Statement statement, DataDefinitionTask<T> task, List<Expression> parameters)
Expand Down
Expand Up @@ -684,7 +684,7 @@ public PhysicalOperation visitExplainAnalyze(ExplainAnalyzeNode node, LocalExecu
context.getNextOperatorId(), context.getNextOperatorId(),
node.getId(), node.getId(),
queryPerformanceFetcher.get(), queryPerformanceFetcher.get(),
metadata, metadata.getFunctionRegistry(),
statsCalculator, statsCalculator,
costCalculator, costCalculator,
node.isVerbose()); node.isVerbose());
Expand Down

0 comments on commit d45566e

Please sign in to comment.