Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add property to enable eager building of plan as part of new native plan checker SPI #23649

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions presto-docs/src/main/sphinx/admin/properties.rst
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,18 @@ output data set is not skewed in order to avoid the overhead of hashing and
redistributing all the data across the network. This can also be specified
on a per-query basis using the ``redistribute_writes`` session property.

``query-execution-fail-fast-enabled``
^^^^^^^^^^^^^^^^^^^^^^^

* **Type:** ``boolean``
* **Default value:** ``false``

This property enables the eager building and validation of a logical plan.
When enabled, the logical plan can be built and validated before execution
begins so that any errors or incompatibilities in the plan will cause the
query to fail before resources are allocated and quickly inform the user
of the error.

.. _tuning-memory:

Memory Management Properties
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,7 @@ public final class SystemSessionProperties
public static final String REWRITE_EXPRESSION_WITH_CONSTANT_EXPRESSION = "rewrite_expression_with_constant_expression";
public static final String PRINT_ESTIMATED_STATS_FROM_CACHE = "print_estimated_stats_from_cache";
public static final String REMOVE_CROSS_JOIN_WITH_CONSTANT_SINGLE_ROW_INPUT = "remove_cross_join_with_constant_single_row_input";
public static final String QUERY_EXECUTION_FAIL_FAST_ENABLED = "query_execution_fail_fast_enabled";

// TODO: Native execution related session properties that are temporarily put here. They will be relocated in the future.
public static final String NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED = "native_simplified_expression_evaluation_enabled";
Expand Down Expand Up @@ -2010,6 +2011,11 @@ public SystemSessionProperties(
"If one input of the cross join is a single row with constant value, remove this cross join and replace with a project node",
featuresConfig.isRemoveCrossJoinWithSingleConstantRow(),
false),
booleanProperty(
QUERY_EXECUTION_FAIL_FAST_ENABLED,
"Enable eager building and validation of logical plan before execution",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"Enable eager building and validation of logical plan before execution",
"Enable eager building and validation of logical plan before queueing",

featuresConfig.isQueryExecutionFailFastEnabled(),
false),
new PropertyMetadata<>(
DEFAULT_VIEW_SECURITY_MODE,
format("Set default view security mode. Options are: %s",
Expand Down Expand Up @@ -3336,6 +3342,11 @@ public static boolean isRewriteExpressionWithConstantEnabled(Session session)
return session.getSystemProperty(REWRITE_EXPRESSION_WITH_CONSTANT_EXPRESSION, Boolean.class);
}

public static boolean isQueryExecutionFailFastEnabled(Session session)
{
return session.getSystemProperty(QUERY_EXECUTION_FAIL_FAST_ENABLED, Boolean.class);
}

public static CreateView.Security getDefaultViewSecurityMode(Session session)
{
return session.getSystemProperty(DEFAULT_VIEW_SECURITY_MODE, CreateView.Security.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
Expand All @@ -83,6 +84,7 @@
import static com.facebook.presto.SystemSessionProperties.getExecutionPolicy;
import static com.facebook.presto.SystemSessionProperties.getQueryAnalyzerTimeout;
import static com.facebook.presto.SystemSessionProperties.isLogInvokedFunctionNamesEnabled;
import static com.facebook.presto.SystemSessionProperties.isQueryExecutionFailFastEnabled;
import static com.facebook.presto.SystemSessionProperties.isSpoolingOutputBufferEnabled;
import static com.facebook.presto.common.RuntimeMetricName.FRAGMENT_PLAN_TIME_NANOS;
import static com.facebook.presto.common.RuntimeMetricName.GET_CANONICAL_INFO_TIME_NANOS;
Expand Down Expand Up @@ -141,6 +143,7 @@ public class SqlQueryExecution
private final PlanCanonicalInfoProvider planCanonicalInfoProvider;
private final QueryAnalysis queryAnalysis;
private final AnalyzerContext analyzerContext;
private final CompletableFuture<PlanRoot> planFuture;

private SqlQueryExecution(
QueryAnalyzer queryAnalyzer,
Expand Down Expand Up @@ -243,6 +246,9 @@ private SqlQueryExecution(
}
}
}

// Optionally build and validate plan immediately, before execution begins
planFuture = isQueryExecutionFailFastEnabled(getSession()) ? createLogicalPlanAsync() : null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm wondering if we can just inline the createLogicalPlanAsync method, and instead, switch the executor based on the session property.

  • If the flag is disabled, use MoreExecutors.sameThreadExecutor()
  • If the flag is enabled, use a new executor which is injected into this class, and which we have metrics for. Note we define the executor currently used HERE, we could define a new one with more restricted paralellism, and which we could add dedicated metrics for (using a QueryExecutionMBean as you see in the link). One thing we'd need to figure out is how to start the planner in the current thread if the queue were full, without a race, but I think this should be doable.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When flag is enabled, sure I can add a new executor

For when it's disabled, IIUC, using something like MoreExecutors.directExecutor() would cause the plan to be validated immediately in the constructor, which would change the default behavior - I don't think we want to do this. I could still inline createLogicalPlanAsync() if you want to make it more compact?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good point, we wouldn't want that! I suppose it should be as you said, if the flag is enabled, submit it to the new executor with restricted parallelism. When we get to line 470, we should try to cancel it without interrupting it--if the cancellation succeeded or if the feature is not enabled, do it in the current thread, otherwise, get() the future.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So when you say try to cancel the future at line 470, is it just in case the ExecutorService hasn't started running the thread, and doing it the current thread would be quicker?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, both quicker, and also it won't be subject to the concurrency restraints of the executor it would be running under.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also, let's make the executor service parallelism configurable.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, good idea

}
}

Expand Down Expand Up @@ -461,7 +467,7 @@ public void start()
timeoutThreadExecutor,
getQueryAnalyzerTimeout(getSession()))) {
// create logical plan for the query
plan = createLogicalPlanAndOptimize();
plan = planFuture == null ? createLogicalPlanAndOptimize() : planFuture.get();
}

metadata.beginQuery(getSession(), plan.getConnectors());
Expand Down Expand Up @@ -528,6 +534,11 @@ public void addFinalQueryInfoListener(StateChangeListener<QueryInfo> stateChange
stateMachine.addQueryInfoStateChangeListener(stateChangeListener);
}

private CompletableFuture<PlanRoot> createLogicalPlanAsync()
{
return CompletableFuture.supplyAsync(this::createLogicalPlanAndOptimize, this.queryExecutor);
}

private PlanRoot createLogicalPlanAndOptimize()
{
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,8 @@ public class FeaturesConfig

private boolean isInlineProjectionsOnValuesEnabled;

private boolean queryExecutionFailFastEnabled;

public enum PartitioningPrecisionStrategy
{
// Let Presto decide when to repartition
Expand Down Expand Up @@ -2969,4 +2971,17 @@ public FeaturesConfig setInlineProjectionsOnValues(boolean isInlineProjectionsOn
this.isInlineProjectionsOnValuesEnabled = isInlineProjectionsOnValuesEnabled;
return this;
}

@Config("query-execution-fail-fast-enabled")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe call this validate-plan-before-queueing? Is that what this is for?

Copy link
Author

@BryanCutler BryanCutler Sep 16, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, it could be named better. However, it isn't specific for only queries that have been queued. It starts building/validating the plan as soon as SqlQueryExecution is constructed instead of waiting until SqlQueryExecution.start() is invoked, when ManagedQueryExecution.startWaitingForResources() has already completed.

Perhaps something like eagerly-validate-plan-before-resources ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIUC, I think queueing will be a more accurate term, because checking for resources happens after queueing.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, my point was that not all queries end up on the queue and might be started right away. In that case this property would still be used and could validate before waiting for resources completes. So I didn't want it to sound like this is only useful if workload has heavy queuing, but if validate-plan-before-queueing is the consensus, it's fine with me.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just explaining my thinking a bit here, rather than thinking of queueing as something that happens to a query, I think of it as a component in Presto. But I see your point as this phrase may not be familiar to folks who are not Presto developers.

What about eager-plan-validation-enabled, and in the documentation we describe it more precisely?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

eager-plan-validation-enabled sgtm

@ConfigDescription("Enable eager building and validation of logical plan before execution")
public FeaturesConfig setQueryExecutionFailFastEnabled(boolean queryExecutionFailFastEnabled)
{
this.queryExecutionFailFastEnabled = queryExecutionFailFastEnabled;
return this;
}

public boolean isQueryExecutionFailFastEnabled()
{
return this.queryExecutionFailFastEnabled;
}
}
Loading