-
Notifications
You must be signed in to change notification settings - Fork 9
implement distributed EXPLAIN ANALYZE #182
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
base: main
Are you sure you want to change the base?
Conversation
This change adds support for displaying a distributed EXPLAIN ANALYZE output. It updates the TPCH validation tests to assert the EXPLAIN ANALYZE output for each query. Implemenation notes: - Adds `src/explain.rs` to stores the main entrypoint to rendering the output string - I left a TODO about pushing some of the work to `DistributedExec` or a new node type - Adds a `Option<DisplayCtx>` field to `DistributedExec` to contain extra information for display purposes. - We use this to smuggle the information into `display_plan_ascii` because its only relevant in the distributed case - Then, at display-time, when displaying a task, we re-write each task plan to use the metrics from the `DislplayCtx` Informs: #123 Remaning work: - disable any metrics propagation if not running EXPLAIN ANALYZE as it adds extra overhead - consider refactoring explain.rs - graphviz - add docs + exalidraw to explain the metrics protocol
7ffad58
to
0b95862
Compare
} | ||
|
||
/// Returns a special stage key used to identify the root "stage" of the distributed plan. | ||
/// TODO: reconcile this with display_plan_graphviz |
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.
Looking for thoughts here.
In display_plan_graphviz, we convert the DistributedExec
to a stage for simplicity where that stage has a different stage id and query id. Maybe they should be on the same page, I'm not sure.
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.
🤔 we probably should not be converting it to a stage neither here or there, as it's not really a stage...
.to_string()), | ||
Some(dist_exec) => { | ||
// If the plan was distributed, collect metrics from the coordinating stage exec. | ||
// TODO: Should we move this into the DistributedExec itself or a new ExplainAnalyzeExec? |
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.
Looking for thoughts here as well
│ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus], metrics=[output_rows=<metric>, elapsed_compute=<metric>] | ||
│ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=<metric>, elapsed_compute=<metric>] | ||
│ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5], metrics=[output_rows=<metric>, elapsed_compute=<metric>] | ||
│ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] |
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.
Metrics is empty, which is fine. One day, we can update the PartitionIsolatorExec
to collect metrics.
Should we have this only render PartitionIsolatorExec: t0:[p0,p1,__,__,__,__]
instead of all the tasks?
└────────────────────────────────────────────────── | ||
┌───── Stage 2 ── Task t0:[p0,p1,p2,p3,p4,p5] | ||
│ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=<metric>, elapsed_compute=<metric>, spill_count=<metric>, spilled_bytes=<metric>, spilled_rows=<metric>, batches_split=<metric>] | ||
│ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order], metrics=[output_rows=<metric>, elapsed_compute=<metric>] |
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.
The metrics=[output_rows=<metric>, elapsed_compute=<metric>]
should they have values there instead of <metric>?
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.
I've hidden them in the insta snapshot config because certain ones change between runs (ex. ones that measure time).
I could keep some, like output_rows. I think those stay the same.
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.
Looking good!
I think we are missing one very important piece of metrics collection: allowing users to manually traverse the plan and doing ExecutionPlan.metrics()
on all the nodes. Here is a practical use case:
Users might have their own custom ExecutionPlan
implementations, and there, they might be collecting their own user-defined metrics. If that's the case, it's very likely that they want to just programmatically traverse the plan looking for their own custom nodes and extract the raw collected metrics values in order to report them as fields in their logs or traces.
This means that unfortunately just being able to display a string with the plan enriched with metrics is not enough for bringing feature parity with what DataFusion offers, and my bet is that if we want to satisfy the "walk your plan and collect your metrics programmatically" case, the approach here will probably need to change.
/// [ExecutionPlan] that executes the inner plan in distributed mode. | ||
/// [ExecutionPan] that executes the inner plan in distributed mode. |
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.
pan == 🍞 in spanish
} | ||
|
||
/// Returns a special stage key used to identify the root "stage" of the distributed plan. | ||
/// TODO: reconcile this with display_plan_graphviz |
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.
🤔 we probably should not be converting it to a stage neither here or there, as it's not really a stage...
return last.and_then(|el| collect_and_create_metrics_flight_data(key, plan, el)); | ||
} | ||
last.and_then(|el| collect_and_create_metrics_flight_data(key, plan, el)) | ||
last |
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.
🤔 thinking about it, we can be in situations where this is never sent.
DataFusion, under certain situations, might decide to abandon some streams (early finishes do to LIMIT X
being reached). In those scenarios, I don't think we are correctly decreasing by 1 the num_partitions_remaining
, we just decrease if we successfully exhaust the full stream.
If this happens, num_partitions_remaining
is never going to reach 0, and this code will never kick in.
Doesn't look like a problem with this specific PR though...
This change adds support for displaying a distributed EXPLAIN ANALYZE output. It updates the TPCH
validation tests to assert the EXPLAIN ANALYZE output for each query.
Implemenation notes:
src/explain.rs
to stores the main entrypoint to rendering the output stringDistributedExec
or a new node typeOption<DisplayCtx>
field toDistributedExec
to contain extra information for display purposes.display_plan_ascii
because its only relevant in thedistributed case
from the
DislplayCtx
Informs: #123
Remaning work: