-
Notifications
You must be signed in to change notification settings - Fork 5.3k
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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; | ||
|
@@ -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; | ||
|
@@ -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, | ||
|
@@ -243,6 +246,9 @@ private SqlQueryExecution( | |
} | ||
} | ||
} | ||
|
||
// Optionally build and validate plan immediately, before execution begins | ||
planFuture = isQueryExecutionFailFastEnabled(getSession()) ? createLogicalPlanAsync() : null; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm wondering if we can just inline the
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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, There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. also, let's make the executor service parallelism configurable. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok, good idea |
||
} | ||
} | ||
|
||
|
@@ -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()); | ||
|
@@ -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 { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -296,6 +296,8 @@ public class FeaturesConfig | |
|
||
private boolean isInlineProjectionsOnValuesEnabled; | ||
|
||
private boolean queryExecutionFailFastEnabled; | ||
|
||
public enum PartitioningPrecisionStrategy | ||
{ | ||
// Let Presto decide when to repartition | ||
|
@@ -2969,4 +2971,17 @@ public FeaturesConfig setInlineProjectionsOnValues(boolean isInlineProjectionsOn | |
this.isInlineProjectionsOnValuesEnabled = isInlineProjectionsOnValuesEnabled; | ||
return this; | ||
} | ||
|
||
@Config("query-execution-fail-fast-enabled") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe call this There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 Perhaps something like There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
@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; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.